diff --git a/.github/PULL_REQUEST_TEMPLATE b/.github/PULL_REQUEST_TEMPLATE index be57f007abbf0..69450add03882 100644 --- a/.github/PULL_REQUEST_TEMPLATE +++ b/.github/PULL_REQUEST_TEMPLATE @@ -6,6 +6,8 @@ Thanks for sending a pull request! Here are some tips for you: 4. Be sure to keep the PR description updated to reflect all changes. 5. Please write your PR title to summarize what this PR proposes. 6. If possible, provide a concise example to reproduce the issue for a faster review. + 7. If you want to add a new configuration, please read the guideline first for naming configurations in + 'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'. --> ### What changes were proposed in this pull request? diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index d53119ad75599..632a0f1aa91fd 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -117,3 +117,40 @@ jobs: run: ./R/install-dev.sh - name: lint-r run: ./dev/lint-r + + docs: + runs-on: ubuntu-latest + name: Generate documents + steps: + - uses: actions/checkout@master + - uses: actions/cache@v1 + with: + path: ~/.m2/repository + key: docs-maven-repo-${{ hashFiles('**/pom.xml') }} + restore-keys: | + docs-maven-repo- + - uses: actions/setup-java@v1 + with: + java-version: '1.8' + - uses: actions/setup-python@v1 + with: + python-version: '3.x' + architecture: 'x64' + - uses: actions/setup-ruby@v1 + with: + ruby-version: '2.7' + - name: Install R + run: | + echo 'deb https://cloud.r-project.org/bin/linux/ubuntu bionic-cran35/' | sudo tee -a /etc/apt/sources.list + curl -sL "https://keyserver.ubuntu.com/pks/lookup?op=get&search=0xE298A3A825C0D65DFD57CBB651716619E084DAB9" | sudo apt-key add + sudo apt-get update + sudo apt-get install -y r-base r-base-dev libcurl4-openssl-dev pandoc + - name: Install packages + run: | + pip install sphinx mkdocs numpy + gem install jekyll jekyll-redirect-from rouge + sudo Rscript -e "install.packages(c('curl', 'xml2', 'httr', 'devtools', 'testthat', 'knitr', 'rmarkdown', 'roxygen2', 'e1071', 'survival'), repos='https://cloud.r-project.org/')" + - name: Run jekyll build + run: | + cd docs + jekyll build diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index c8cb1c3a992ad..aa51b98fa3c58 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,6 +1,6 @@ Package: SparkR Type: Package -Version: 3.0.0 +Version: 3.1.0 Title: R Front End for 'Apache Spark' Description: Provides an R Front end for 'Apache Spark' . Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 7ed2e36d59531..2f7b876f0ec33 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -199,9 +199,13 @@ exportMethods("%<=>%", "approx_count_distinct", "approxCountDistinct", "approxQuantile", + "array_aggregate", "array_contains", "array_distinct", "array_except", + "array_exists", + "array_filter", + "array_forall", "array_intersect", "array_join", "array_max", @@ -210,9 +214,11 @@ exportMethods("%<=>%", "array_remove", "array_repeat", "array_sort", + "array_transform", "arrays_overlap", "array_union", "arrays_zip", + "arrays_zip_with", "asc", "ascii", "asin", @@ -314,10 +320,12 @@ exportMethods("%<=>%", "ltrim", "map_concat", "map_entries", + "map_filter", "map_from_arrays", "map_from_entries", "map_keys", "map_values", + "map_zip_with", "max", "md5", "mean", @@ -396,6 +404,8 @@ exportMethods("%<=>%", "to_timestamp", "to_utc_timestamp", "translate", + "transform_keys", + "transform_values", "trim", "trunc", "unbase64", diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index 48f69d5769620..0ecf688a636d1 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -219,6 +219,34 @@ NULL #' the DDL-formatted string literal can also be accepted. #' \item \code{from_csv}: a structType object, DDL-formatted string or \code{schema_of_csv} #' } +#' +#' @param f a \code{function} mapping from \code{Column(s)} to \code{Column}. +#' \itemize{ +#' \item \code{array_exists} +#' \item \code{array_filter} the Boolean \code{function} used to filter the data. +#' Either unary or binary. In the latter case the second argument +#' is the index in the array (0-based). +#' \item \code{array_forall} the Boolean unary \code{function} used to filter the data. +#' \item \code{array_transform} a \code{function} used to transform the data. +#' Either unary or binary. In the latter case the second argument +#' is the index in the array (0-based). +#' \item \code{arrays_zip_with} +#' \item \code{map_zip_with} +#' \item \code{map_filter} the Boolean binary \code{function} used to filter the data. +#' The first argument is the key, the second argument is the value. +#' \item \code{transform_keys} a binary \code{function} +#' used to transform the data. The first argument is the key, the second argument +#' is the value. +#' \item \code{transform_values} a binary \code{function} +#' used to transform the data. The first argument is the key, the second argument +#' is the value. +#' } +#' @param zero a \code{Column} used as the initial value in \code{array_aggregate} +#' @param merge a \code{function} a binary function \code{(Column, Column) -> Column} +#' used in \code{array_aggregate}to merge values (the second argument) +#' into accumulator (the first argument). +#' @param finish an unary \code{function} \code{(Column) -> Column} used to +#' apply final transformation on the accumulated data in \code{array_aggregate}. #' @param ... additional argument(s). #' \itemize{ #' \item \code{to_json}, \code{from_json} and \code{schema_of_json}: this contains @@ -244,6 +272,14 @@ NULL #' head(select(tmp, array_max(tmp$v1), array_min(tmp$v1), array_distinct(tmp$v1))) #' head(select(tmp, array_position(tmp$v1, 21), array_repeat(df$mpg, 3), array_sort(tmp$v1))) #' head(select(tmp, reverse(tmp$v1), array_remove(tmp$v1, 21))) +#' head(select(tmp, array_transform("v1", function(x) x * 10))) +#' head(select(tmp, array_exists("v1", function(x) x > 120))) +#' head(select(tmp, array_forall("v1", function(x) x >= 8.0))) +#' head(select(tmp, array_filter("v1", function(x) x < 10))) +#' head(select(tmp, array_aggregate("v1", lit(0), function(acc, y) acc + y))) +#' head(select( +#' tmp, +#' array_aggregate("v1", lit(0), function(acc, y) acc + y, function(acc) acc / 10))) #' tmp2 <- mutate(tmp, v2 = explode(tmp$v1)) #' head(tmp2) #' head(select(tmp, posexplode(tmp$v1))) @@ -253,17 +289,22 @@ NULL #' tmp3 <- mutate(df, v3 = create_map(df$model, df$cyl)) #' head(select(tmp3, map_entries(tmp3$v3), map_keys(tmp3$v3), map_values(tmp3$v3))) #' head(select(tmp3, element_at(tmp3$v3, "Valiant"), map_concat(tmp3$v3, tmp3$v3))) +#' head(select(tmp3, transform_keys("v3", function(k, v) upper(k)))) +#' head(select(tmp3, transform_values("v3", function(k, v) v * 10))) +#' head(select(tmp3, map_filter("v3", function(k, v) v < 42))) #' tmp4 <- mutate(df, v4 = create_array(df$mpg, df$cyl), v5 = create_array(df$cyl, df$hp)) #' head(select(tmp4, concat(tmp4$v4, tmp4$v5), arrays_overlap(tmp4$v4, tmp4$v5))) #' head(select(tmp4, array_except(tmp4$v4, tmp4$v5), array_intersect(tmp4$v4, tmp4$v5))) #' head(select(tmp4, array_union(tmp4$v4, tmp4$v5))) #' head(select(tmp4, arrays_zip(tmp4$v4, tmp4$v5))) #' head(select(tmp, concat(df$mpg, df$cyl, df$hp))) +#' head(select(tmp4, arrays_zip_with(tmp4$v4, tmp4$v5, function(x, y) x * y))) #' tmp5 <- mutate(df, v6 = create_array(df$model, df$model)) #' head(select(tmp5, array_join(tmp5$v6, "#"), array_join(tmp5$v6, "#", "NULL"))) #' tmp6 <- mutate(df, v7 = create_array(create_array(df$model, df$model))) #' head(select(tmp6, flatten(tmp6$v7))) #' tmp7 <- mutate(df, v8 = create_array(df$model, df$cyl), v9 = create_array(df$model, df$hp)) +#' head(select(tmp7, arrays_zip_with("v8", "v9", function(x, y) (x * y) %% 3))) #' head(select(tmp7, map_from_arrays(tmp7$v8, tmp7$v9))) #' tmp8 <- mutate(df, v10 = create_array(struct(df$model, df$cyl))) #' head(select(tmp8, map_from_entries(tmp8$v10)))} @@ -3281,6 +3322,121 @@ setMethod("row_number", ###################### Collection functions###################### +#' Create o.a.s.sql.expressions.UnresolvedNamedLambdaVariable, +#' convert it to o.s.sql.Column and wrap with R Column. +#' Used by higher order functions. +#' +#' @param ... character of length = 1 +#' if length(...) > 1 then argument is interpreted as a nested +#' Column, for example \code{unresolved_named_lambda_var("a", "b", "c")} +#' yields unresolved \code{a.b.c} +#' @return Column object wrapping JVM UnresolvedNamedLambdaVariable +unresolved_named_lambda_var <- function(...) { + jc <- newJObject( + "org.apache.spark.sql.Column", + newJObject( + "org.apache.spark.sql.catalyst.expressions.UnresolvedNamedLambdaVariable", + list(...) + ) + ) + column(jc) +} + +#' Create o.a.s.sql.expressions.LambdaFunction corresponding +#' to transformation described by func. +#' Used by higher order functions. +#' +#' @param fun R \code{function} (unary, binary or ternary) +#' that transforms \code{Columns} into a \code{Column} +#' @return JVM \code{LambdaFunction} object +create_lambda <- function(fun) { + as_jexpr <- function(x) callJMethod(x@jc, "expr") + + # Process function arguments + parameters <- formals(fun) + nparameters <- length(parameters) + + stopifnot( + nparameters >= 1 & + nparameters <= 3 & + !"..." %in% names(parameters) + ) + + args <- lapply(c("x", "y", "z")[seq_along(parameters)], function(p) { + unresolved_named_lambda_var(p) + }) + + # Invoke function and validate return type + result <- do.call(fun, args) + stopifnot(class(result) == "Column") + + # Convert both Columns to Scala expressions + jexpr <- as_jexpr(result) + + jargs <- handledCallJStatic( + "org.apache.spark.api.python.PythonUtils", + "toSeq", + handledCallJStatic( + "java.util.Arrays", "asList", lapply(args, as_jexpr) + ) + ) + + # Create Scala LambdaFunction + newJObject( + "org.apache.spark.sql.catalyst.expressions.LambdaFunction", + jexpr, + jargs, + FALSE + ) +} + +#' Invokes higher order function expression identified by name, +#' (relative to o.a.s.sql.catalyst.expressions) +#' +#' @param name character +#' @param cols list of character or Column objects +#' @param funs list of named list(fun = ..., expected_narg = ...) +#' @return a \code{Column} representing name applied to cols with funs +invoke_higher_order_function <- function(name, cols, funs) { + as_jexpr <- function(x) { + if (class(x) == "character") { + x <- column(x) + } + callJMethod(x@jc, "expr") + } + + jexpr <- do.call(newJObject, c( + paste("org.apache.spark.sql.catalyst.expressions", name, sep = "."), + lapply(cols, as_jexpr), + lapply(funs, create_lambda) + )) + + column(newJObject("org.apache.spark.sql.Column", jexpr)) +} + +#' @details +#' \code{array_aggregate} Applies a binary operator to an initial state +#' and all elements in the array, and reduces this to a single state. +#' The final state is converted into the final result by applying +#' a finish function. +#' +#' @rdname column_collection_functions +#' @aliases array_aggregate array_aggregate,characterOrColumn,Column,function-method +#' @note array_aggregate since 3.1.0 +setMethod("array_aggregate", + signature(x = "characterOrColumn", zero = "Column", merge = "function"), + function(x, zero, merge, finish = NULL) { + invoke_higher_order_function( + "ArrayAggregate", + cols = list(x, zero), + funs = if (is.null(finish)) { + list(merge) + } else { + list(merge, finish) + } + ) + }) + #' @details #' \code{array_contains}: Returns null if the array is null, true if the array contains #' the value, and false otherwise. @@ -3322,6 +3478,54 @@ setMethod("array_except", column(jc) }) +#' @details +#' \code{array_exists} Returns whether a predicate holds for one or more elements in the array. +#' +#' @rdname column_collection_functions +#' @aliases array_exists array_exists,characterOrColumn,function-method +#' @note array_exists since 3.1.0 +setMethod("array_exists", + signature(x = "characterOrColumn", f = "function"), + function(x, f) { + invoke_higher_order_function( + "ArrayExists", + cols = list(x), + funs = list(f) + ) + }) + +#' @details +#' \code{array_filter} Returns an array of elements for which a predicate holds in a given array. +#' +#' @rdname column_collection_functions +#' @aliases array_filter array_filter,characterOrColumn,function-method +#' @note array_filter since 3.1.0 +setMethod("array_filter", + signature(x = "characterOrColumn", f = "function"), + function(x, f) { + invoke_higher_order_function( + "ArrayFilter", + cols = list(x), + funs = list(f) + ) + }) + +#' @details +#' \code{array_forall} Returns whether a predicate holds for every element in the array. +#' +#' @rdname column_collection_functions +#' @aliases array_forall array_forall,characterOrColumn,function-method +#' @note array_forall since 3.1.0 +setMethod("array_forall", + signature(x = "characterOrColumn", f = "function"), + function(x, f) { + invoke_higher_order_function( + "ArrayForAll", + cols = list(x), + funs = list(f) + ) + }) + #' @details #' \code{array_intersect}: Returns an array of the elements in the intersection of the given two #' arrays, without duplicates. @@ -3446,6 +3650,23 @@ setMethod("array_sort", column(jc) }) +#' @details +#' \code{array_transform} Returns an array of elements after applying +#' a transformation to each element in the input array. +#' +#' @rdname column_collection_functions +#' @aliases array_transform array_transform,characterOrColumn,characterOrColumn,function-method +#' @note array_transform since 3.1.0 +setMethod("array_transform", + signature(x = "characterOrColumn", f = "function"), + function(x, f) { + invoke_higher_order_function( + "ArrayTransform", + cols = list(x), + funs = list(f) + ) + }) + #' @details #' \code{arrays_overlap}: Returns true if the input arrays have at least one non-null element in #' common. If not and both arrays are non-empty and any of them contains a null, it returns null. @@ -3493,6 +3714,24 @@ setMethod("arrays_zip", column(jc) }) +#' @details +#' \code{arrays_zip_with} Merge two given arrays, element-wise, into a single array +#' using a function. If one array is shorter, nulls are appended at the end +#' to match the length of the longer array, before applying the function. +#' +#' @rdname column_collection_functions +#' @aliases arrays_zip_with arrays_zip_with,characterOrColumn,characterOrColumn,function-method +#' @note zip_with since 3.1.0 +setMethod("arrays_zip_with", + signature(x = "characterOrColumn", y = "characterOrColumn", f = "function"), + function(x, y, f) { + invoke_higher_order_function( + "ZipWith", + cols = list(x, y), + funs = list(f) + ) + }) + #' @details #' \code{shuffle}: Returns a random permutation of the given array. #' @@ -3550,6 +3789,21 @@ setMethod("map_entries", column(jc) }) +#' @details +#' \code{map_filter} Returns a map whose key-value pairs satisfy a predicate. +#' +#' @rdname column_collection_functions +#' @aliases map_filter map_filter,characterOrColumn,function-method +#' @note map_filter since 3.1.0 +setMethod("map_filter", + signature(x = "characterOrColumn", f = "function"), + function(x, f) { + invoke_higher_order_function( + "MapFilter", + cols = list(x), + funs = list(f)) + }) + #' @details #' \code{map_from_arrays}: Creates a new map column. The array in the first column is used for #' keys. The array in the second column is used for values. All elements in the array for key @@ -3591,6 +3845,41 @@ setMethod("map_keys", column(jc) }) +#' @details +#' \code{transform_keys} Applies a function to every key-value pair in a map and returns +#' a map with the results of those applications as the new keys for the pairs. +#' +#' @rdname column_collection_functions +#' @aliases transform_keys transform_keys,characterOrColumn,function-method +#' @note transform_keys since 3.1.0 +setMethod("transform_keys", + signature(x = "characterOrColumn", f = "function"), + function(x, f) { + invoke_higher_order_function( + "TransformKeys", + cols = list(x), + funs = list(f) + ) + }) + +#' @details +#' \code{transform_values} Applies a function to every key-value pair in a map and returns +#' a map with the results of those applications as the new values for the pairs. +#' +#' @rdname column_collection_functions +#' @aliases transform_values transform_values,characterOrColumn,function-method +#' @note transform_values since 3.1.0 +setMethod("transform_values", + signature(x = "characterOrColumn", f = "function"), + function(x, f) { + invoke_higher_order_function( + "TransformValues", + cols = list(x), + funs = list(f) + ) + }) + + #' @details #' \code{map_values}: Returns an unordered array containing the values of the map. #' @@ -3604,6 +3893,24 @@ setMethod("map_values", column(jc) }) +#' @details +#' \code{map_zip} Merge two given maps, key-wise into a single map using a function. +#' +#' @rdname column_collection_functions +#' @aliases map_zip_with map_zip_with,characterOrColumn,characterOrColumn,function-method +#' +#' @examples +#' @note map_zip_with since 3.1.0 +setMethod("map_zip_with", + signature(x = "characterOrColumn", y = "characterOrColumn", f = "function"), + function(x, y, f) { + invoke_higher_order_function( + "MapZipWith", + cols = list(x, y), + funs = list(f) + ) + }) + #' @details #' \code{element_at}: Returns element of array at given index in \code{extraction} if #' \code{x} is array. Returns value for the given key in \code{extraction} if \code{x} is map. diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 4134d5cecc888..a52ec7a4a27c1 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -757,6 +757,10 @@ setGeneric("approx_count_distinct", function(x, ...) { standardGeneric("approx_c #' @name NULL setGeneric("approxCountDistinct", function(x, ...) { standardGeneric("approxCountDistinct") }) +#' @rdname column_collection_functions +#' @name NULL +setGeneric("array_aggregate", function(x, zero, merge, ...) { standardGeneric("array_aggregate") }) + #' @rdname column_collection_functions #' @name NULL setGeneric("array_contains", function(x, value) { standardGeneric("array_contains") }) @@ -769,6 +773,18 @@ setGeneric("array_distinct", function(x) { standardGeneric("array_distinct") }) #' @name NULL setGeneric("array_except", function(x, y) { standardGeneric("array_except") }) +#' @rdname column_collection_functions +#' @name NULL +setGeneric("array_exists", function(x, f) { standardGeneric("array_exists") }) + +#' @rdname column_collection_functions +#' @name NULL +setGeneric("array_forall", function(x, f) { standardGeneric("array_forall") }) + +#' @rdname column_collection_functions +#' @name NULL +setGeneric("array_filter", function(x, f) { standardGeneric("array_filter") }) + #' @rdname column_collection_functions #' @name NULL setGeneric("array_intersect", function(x, y) { standardGeneric("array_intersect") }) @@ -801,6 +817,10 @@ setGeneric("array_repeat", function(x, count) { standardGeneric("array_repeat") #' @name NULL setGeneric("array_sort", function(x) { standardGeneric("array_sort") }) +#' @rdname column_collection_functions +#' @name NULL +setGeneric("array_transform", function(x, f) { standardGeneric("array_transform") }) + #' @rdname column_collection_functions #' @name NULL setGeneric("arrays_overlap", function(x, y) { standardGeneric("arrays_overlap") }) @@ -813,6 +833,10 @@ setGeneric("array_union", function(x, y) { standardGeneric("array_union") }) #' @name NULL setGeneric("arrays_zip", function(x, ...) { standardGeneric("arrays_zip") }) +#' @rdname column_collection_functions +#' @name NULL +setGeneric("arrays_zip_with", function(x, y, f) { standardGeneric("arrays_zip_with") }) + #' @rdname column_string_functions #' @name NULL setGeneric("ascii", function(x) { standardGeneric("ascii") }) @@ -1086,6 +1110,10 @@ setGeneric("map_concat", function(x, ...) { standardGeneric("map_concat") }) #' @name NULL setGeneric("map_entries", function(x) { standardGeneric("map_entries") }) +#' @rdname column_collection_functions +#' @name NULL +setGeneric("map_filter", function(x, f) { standardGeneric("map_filter") }) + #' @rdname column_collection_functions #' @name NULL setGeneric("map_from_arrays", function(x, y) { standardGeneric("map_from_arrays") }) @@ -1102,6 +1130,10 @@ setGeneric("map_keys", function(x) { standardGeneric("map_keys") }) #' @name NULL setGeneric("map_values", function(x) { standardGeneric("map_values") }) +#' @rdname column_collection_functions +#' @name NULL +setGeneric("map_zip_with", function(x, y, f) { standardGeneric("map_zip_with") }) + #' @rdname column_misc_functions #' @name NULL setGeneric("md5", function(x) { standardGeneric("md5") }) @@ -1314,6 +1346,14 @@ setGeneric("substring_index", function(x, delim, count) { standardGeneric("subst #' @name NULL setGeneric("sumDistinct", function(x) { standardGeneric("sumDistinct") }) +#' @rdname column_collection_functions +#' @name NULL +setGeneric("transform_keys", function(x, f) { standardGeneric("transform_keys") }) + +#' @rdname column_collection_functions +#' @name NULL +setGeneric("transform_values", function(x, f) { standardGeneric("transform_values") }) + #' @rdname column_math_functions #' @name NULL setGeneric("degrees", function(x) { standardGeneric("degrees") }) diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R b/R/pkg/tests/fulltests/test_sparkSQL.R index c1d277ac84be1..0f26184fa3e9e 100644 --- a/R/pkg/tests/fulltests/test_sparkSQL.R +++ b/R/pkg/tests/fulltests/test_sparkSQL.R @@ -1994,6 +1994,70 @@ test_that("when(), otherwise() and ifelse() with column on a DataFrame", { expect_equal(collect(select(df, ifelse(df$a > 1 & df$b > 2, lit(0), lit(1))))[, 1], c(1, 0)) }) +test_that("higher order functions", { + df <- select( + createDataFrame(data.frame(id = 1)), + expr("CAST(array(1.0, 2.0, -3.0, -4.0) AS array) xs"), + expr("CAST(array(0.0, 3.0, 48.0) AS array) ys"), + expr("array('FAILED', 'SUCCEDED') as vs"), + expr("map('foo', 1, 'bar', 2) as mx"), + expr("map('foo', 42, 'bar', -1, 'baz', 0) as my") + ) + + map_to_sorted_array <- function(x) { + sort_array(arrays_zip(map_keys(x), map_values(x))) + } + + result <- collect(select( + df, + array_transform("xs", function(x) x + 1) == expr("transform(xs, x -> x + 1)"), + array_transform("xs", function(x, i) otherwise(when(i %% 2 == 0, x), -x)) == + expr("transform(xs, (x, i) -> CASE WHEN ((i % 2.0) = 0.0) THEN x ELSE (- x) END)"), + array_exists("vs", function(v) rlike(v, "FAILED")) == + expr("exists(vs, v -> (v RLIKE 'FAILED'))"), + array_forall("xs", function(x) x > 0) == + expr("forall(xs, x -> x > 0)"), + array_filter("xs", function(x, i) x > 0 | i %% 2 == 0) == + expr("filter(xs, (x, i) -> x > 0 OR i % 2 == 0)"), + array_filter("xs", function(x) signum(x) > 0) == + expr("filter(xs, x -> signum(x) > 0)"), + array_aggregate("xs", lit(0.0), function(x, y) otherwise(when(x > y, x), y)) == + expr("aggregate(xs, CAST(0.0 AS double), (x, y) -> CASE WHEN x > y THEN x ELSE y END)"), + array_aggregate( + "xs", + struct( + alias(lit(0.0), "count"), + alias(lit(0.0), "sum") + ), + function(acc, x) { + count <- getItem(acc, "count") + sum <- getItem(acc, "sum") + struct(alias(count + 1.0, "count"), alias(sum + x, "sum")) + }, + function(acc) getItem(acc, "sum") / getItem(acc, "count") + ) == expr(paste0( + "aggregate(xs, struct(CAST(0.0 AS double) count, CAST(0.0 AS double) sum), ", + "(acc, x) -> ", + "struct(cast(acc.count + 1.0 AS double) count, CAST(acc.sum + x AS double) sum), ", + "acc -> acc.sum / acc.count)" + )), + arrays_zip_with("xs", "ys", function(x, y) x + y) == + expr("zip_with(xs, ys, (x, y) -> x + y)"), + map_to_sorted_array(transform_keys("mx", function(k, v) upper(k))) == + map_to_sorted_array(expr("transform_keys(mx, (k, v) -> upper(k))")), + map_to_sorted_array(transform_values("mx", function(k, v) v * 2)) == + map_to_sorted_array(expr("transform_values(mx, (k, v) -> v * 2)")), + map_to_sorted_array(map_filter(column("my"), function(k, v) lower(v) != "foo")) == + map_to_sorted_array(expr("map_filter(my, (k, v) -> lower(v) != 'foo')")), + map_to_sorted_array(map_zip_with("mx", "my", function(k, vx, vy) vx * vy)) == + map_to_sorted_array(expr("map_zip_with(mx, my, (k, vx, vy) -> vx * vy)")) + )) + + expect_true(all(unlist(result))) + + expect_error(array_transform("xs", function(...) 42)) +}) + test_that("group by, agg functions", { df <- read.json(jsonPath) df1 <- agg(df, name = "max", age = "sum") diff --git a/appveyor.yml b/appveyor.yml index 5d98260265b1a..fc0b7d53ddabc 100644 --- a/appveyor.yml +++ b/appveyor.yml @@ -42,8 +42,7 @@ install: # Install maven and dependencies - ps: .\dev\appveyor-install-dependencies.ps1 # Required package for R unit tests - - cmd: R -e "install.packages(c('knitr', 'rmarkdown', 'e1071', 'survival', 'arrow'), repos='https://cloud.r-project.org/')" - - cmd: R -e "install.packages(c('crayon', 'praise', 'R6', 'testthat'), repos='https://cloud.r-project.org/')" + - cmd: R -e "install.packages(c('knitr', 'rmarkdown', 'testthat', 'e1071', 'survival', 'arrow'), repos='https://cloud.r-project.org/')" - cmd: R -e "packageVersion('knitr'); packageVersion('rmarkdown'); packageVersion('testthat'); packageVersion('e1071'); packageVersion('survival'); packageVersion('arrow')" build_script: diff --git a/assembly/pom.xml b/assembly/pom.xml index 193ad3d671bcf..d17abe857ade5 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.1.0-SNAPSHOT ../pom.xml diff --git a/bin/pyspark b/bin/pyspark index 44891aee2e0a3..ad4132fb59eb0 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -50,7 +50,7 @@ export PYSPARK_DRIVER_PYTHON_OPTS # Add the PySpark classes to the Python path: export PYTHONPATH="${SPARK_HOME}/python/:$PYTHONPATH" -export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.8.1-src.zip:$PYTHONPATH" +export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.9-src.zip:$PYTHONPATH" # Load the PySpark shell.py script when ./pyspark is used interactively: export OLD_PYTHONSTARTUP="$PYTHONSTARTUP" diff --git a/bin/pyspark2.cmd b/bin/pyspark2.cmd index 479fd464c7d3e..dc34be1a41706 100644 --- a/bin/pyspark2.cmd +++ b/bin/pyspark2.cmd @@ -30,7 +30,7 @@ if "x%PYSPARK_DRIVER_PYTHON%"=="x" ( ) set PYTHONPATH=%SPARK_HOME%\python;%PYTHONPATH% -set PYTHONPATH=%SPARK_HOME%\python\lib\py4j-0.10.8.1-src.zip;%PYTHONPATH% +set PYTHONPATH=%SPARK_HOME%\python\lib\py4j-0.10.9-src.zip;%PYTHONPATH% set OLD_PYTHONSTARTUP=%PYTHONSTARTUP% set PYTHONSTARTUP=%SPARK_HOME%\python\pyspark\shell.py diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index a1c8a8e6582eb..39cdc6d6d6cd3 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.1.0-SNAPSHOT ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 163c250054e4d..9d5bc9aae0719 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.1.0-SNAPSHOT ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index a6d99813a8501..00f1defbb0093 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.1.0-SNAPSHOT ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 76a402bb2bd31..0225db81925c5 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.1.0-SNAPSHOT ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 3c3c0d2d96a1c..72a2c4ceb43b6 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.1.0-SNAPSHOT ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 883b73a69c9de..ea16dadca40cb 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.1.0-SNAPSHOT ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 93a4f67fd23f2..769e2518b1fd4 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.1.0-SNAPSHOT ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 9d54d21b95ba3..b0f68880f1d8a 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.0.0-SNAPSHOT + 3.1.0-SNAPSHOT ../pom.xml @@ -414,7 +414,7 @@ net.sf.py4j py4j - 0.10.8.1 + 0.10.9 org.apache.spark diff --git a/core/src/main/resources/org/apache/spark/ui/static/streaming-page.js b/core/src/main/resources/org/apache/spark/ui/static/streaming-page.js index 5b75bc3011b6d..ed3e65c386dce 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/streaming-page.js +++ b/core/src/main/resources/org/apache/spark/ui/static/streaming-page.js @@ -171,7 +171,7 @@ function drawTimeline(id, data, minX, maxX, minY, maxY, unitY, batchInterval) { .attr("cy", function(d) { return y(d.y); }) .attr("r", function(d) { return isFailedBatch(d.x) ? "2" : "3";}) .on('mouseover', function(d) { - var tip = formatYValue(d.y) + " " + unitY + " at " + timeFormat[d.x]; + var tip = formatYValue(d.y) + " " + unitY + " at " + timeTipStrings[d.x]; showBootstrapTooltip(d3.select(this).node(), tip); // show the point d3.select(this) diff --git a/core/src/main/resources/org/apache/spark/ui/static/structured-streaming-page.js b/core/src/main/resources/org/apache/spark/ui/static/structured-streaming-page.js index 70250fdbd2d0c..c92226b408b6c 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/structured-streaming-page.js +++ b/core/src/main/resources/org/apache/spark/ui/static/structured-streaming-page.js @@ -106,12 +106,12 @@ function drawAreaStack(id, labels, values, minX, maxX, minY, maxY) { .on('mouseover', function(d) { var tip = ''; var idx = 0; - var _values = timeToValues[d._x] + var _values = formattedTimeToValues[d._x]; _values.forEach(function (k) { tip += labels[idx] + ': ' + k + ' '; idx += 1; }); - tip += " at " + d._x + tip += " at " + formattedTimeTipStrings[d._x]; showBootstrapTooltip(d3.select(this).node(), tip); }) .on('mouseout', function() { diff --git a/core/src/main/scala/org/apache/spark/BarrierCoordinator.scala b/core/src/main/scala/org/apache/spark/BarrierCoordinator.scala index 4e417679ca663..be5036e82e4b2 100644 --- a/core/src/main/scala/org/apache/spark/BarrierCoordinator.scala +++ b/core/src/main/scala/org/apache/spark/BarrierCoordinator.scala @@ -17,12 +17,17 @@ package org.apache.spark +import java.nio.charset.StandardCharsets.UTF_8 import java.util.{Timer, TimerTask} import java.util.concurrent.ConcurrentHashMap import java.util.function.Consumer import scala.collection.mutable.ArrayBuffer +import org.json4s.JsonAST._ +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods.{compact, render} + import org.apache.spark.internal.Logging import org.apache.spark.rpc.{RpcCallContext, RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.scheduler.{LiveListenerBus, SparkListener, SparkListenerStageCompleted} @@ -99,10 +104,15 @@ private[spark] class BarrierCoordinator( // reset when a barrier() call fails due to timeout. private var barrierEpoch: Int = 0 - // An array of RPCCallContexts for barrier tasks that are waiting for reply of a barrier() - // call. + // An Array of RPCCallContexts for barrier tasks that have made a blocking runBarrier() call private val requesters: ArrayBuffer[RpcCallContext] = new ArrayBuffer[RpcCallContext](numTasks) + // An Array of allGather messages for barrier tasks that have made a blocking runBarrier() call + private val allGatherMessages: ArrayBuffer[String] = new Array[String](numTasks).to[ArrayBuffer] + + // The blocking requestMethod called by tasks to sync up for this stage attempt + private var requestMethodToSync: RequestMethod.Value = RequestMethod.BARRIER + // A timer task that ensures we may timeout for a barrier() call. private var timerTask: TimerTask = null @@ -130,9 +140,32 @@ private[spark] class BarrierCoordinator( // Process the global sync request. The barrier() call succeed if collected enough requests // within a configured time, otherwise fail all the pending requests. - def handleRequest(requester: RpcCallContext, request: RequestToSync): Unit = synchronized { + def handleRequest( + requester: RpcCallContext, + request: RequestToSync + ): Unit = synchronized { val taskId = request.taskAttemptId val epoch = request.barrierEpoch + val requestMethod = request.requestMethod + val partitionId = request.partitionId + val allGatherMessage = request match { + case ag: AllGatherRequestToSync => ag.allGatherMessage + case _ => "" + } + + if (requesters.size == 0) { + requestMethodToSync = requestMethod + } + + if (requestMethodToSync != requestMethod) { + requesters.foreach( + _.sendFailure(new SparkException(s"$barrierId tried to use requestMethod " + + s"`$requestMethod` during barrier epoch $barrierEpoch, which does not match " + + s"the current synchronized requestMethod `$requestMethodToSync`" + )) + ) + cleanupBarrierStage(barrierId) + } // Require the number of tasks is correctly set from the BarrierTaskContext. require(request.numTasks == numTasks, s"Number of tasks of $barrierId is " + @@ -153,6 +186,7 @@ private[spark] class BarrierCoordinator( } // Add the requester to array of RPCCallContexts pending for reply. requesters += requester + allGatherMessages(partitionId) = allGatherMessage logInfo(s"Barrier sync epoch $barrierEpoch from $barrierId received update from Task " + s"$taskId, current progress: ${requesters.size}/$numTasks.") if (maybeFinishAllRequesters(requesters, numTasks)) { @@ -173,7 +207,13 @@ private[spark] class BarrierCoordinator( requesters: ArrayBuffer[RpcCallContext], numTasks: Int): Boolean = { if (requesters.size == numTasks) { - requesters.foreach(_.reply(())) + requestMethodToSync match { + case RequestMethod.BARRIER => + requesters.foreach(_.reply("")) + case RequestMethod.ALL_GATHER => + val json: String = compact(render(allGatherMessages)) + requesters.foreach(_.reply(json)) + } true } else { false @@ -199,11 +239,11 @@ private[spark] class BarrierCoordinator( } override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { - case request @ RequestToSync(numTasks, stageId, stageAttemptId, _, _) => + case request: RequestToSync => // Get or init the ContextBarrierState correspond to the stage attempt. - val barrierId = ContextBarrierId(stageId, stageAttemptId) + val barrierId = ContextBarrierId(request.stageId, request.stageAttemptId) states.computeIfAbsent(barrierId, - (key: ContextBarrierId) => new ContextBarrierState(key, numTasks)) + (key: ContextBarrierId) => new ContextBarrierState(key, request.numTasks)) val barrierState = states.get(barrierId) barrierState.handleRequest(context, request) @@ -216,6 +256,16 @@ private[spark] class BarrierCoordinator( private[spark] sealed trait BarrierCoordinatorMessage extends Serializable +private[spark] sealed trait RequestToSync extends BarrierCoordinatorMessage { + def numTasks: Int + def stageId: Int + def stageAttemptId: Int + def taskAttemptId: Long + def barrierEpoch: Int + def partitionId: Int + def requestMethod: RequestMethod.Value +} + /** * A global sync request message from BarrierTaskContext, by `barrier()` call. Each request is * identified by stageId + stageAttemptId + barrierEpoch. @@ -224,11 +274,44 @@ private[spark] sealed trait BarrierCoordinatorMessage extends Serializable * @param stageId ID of current stage * @param stageAttemptId ID of current stage attempt * @param taskAttemptId Unique ID of current task - * @param barrierEpoch ID of the `barrier()` call, a task may consist multiple `barrier()` calls. + * @param barrierEpoch ID of the `barrier()` call, a task may consist multiple `barrier()` calls + * @param partitionId ID of the current partition the task is assigned to + * @param requestMethod The BarrierTaskContext method that was called to trigger BarrierCoordinator */ -private[spark] case class RequestToSync( - numTasks: Int, - stageId: Int, - stageAttemptId: Int, - taskAttemptId: Long, - barrierEpoch: Int) extends BarrierCoordinatorMessage +private[spark] case class BarrierRequestToSync( + numTasks: Int, + stageId: Int, + stageAttemptId: Int, + taskAttemptId: Long, + barrierEpoch: Int, + partitionId: Int, + requestMethod: RequestMethod.Value +) extends RequestToSync + +/** + * A global sync request message from BarrierTaskContext, by `allGather()` call. Each request is + * identified by stageId + stageAttemptId + barrierEpoch. + * + * @param numTasks The number of global sync requests the BarrierCoordinator shall receive + * @param stageId ID of current stage + * @param stageAttemptId ID of current stage attempt + * @param taskAttemptId Unique ID of current task + * @param barrierEpoch ID of the `barrier()` call, a task may consist multiple `barrier()` calls + * @param partitionId ID of the current partition the task is assigned to + * @param requestMethod The BarrierTaskContext method that was called to trigger BarrierCoordinator + * @param allGatherMessage Message sent from the BarrierTaskContext if requestMethod is ALL_GATHER + */ +private[spark] case class AllGatherRequestToSync( + numTasks: Int, + stageId: Int, + stageAttemptId: Int, + taskAttemptId: Long, + barrierEpoch: Int, + partitionId: Int, + requestMethod: RequestMethod.Value, + allGatherMessage: String +) extends RequestToSync + +private[spark] object RequestMethod extends Enumeration { + val BARRIER, ALL_GATHER = Value +} diff --git a/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala b/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala index 3d369802f3023..2263538a11676 100644 --- a/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala +++ b/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala @@ -17,11 +17,19 @@ package org.apache.spark +import java.nio.charset.StandardCharsets.UTF_8 import java.util.{Properties, Timer, TimerTask} import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer import scala.concurrent.TimeoutException import scala.concurrent.duration._ +import scala.language.postfixOps + +import org.json4s.DefaultFormats +import org.json4s.JsonAST._ +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods.parse import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.executor.TaskMetrics @@ -59,49 +67,31 @@ class BarrierTaskContext private[spark] ( // from different tasks within the same barrier stage attempt to succeed. private lazy val numTasks = getTaskInfos().size - /** - * :: Experimental :: - * Sets a global barrier and waits until all tasks in this stage hit this barrier. Similar to - * MPI_Barrier function in MPI, the barrier() function call blocks until all tasks in the same - * stage have reached this routine. - * - * CAUTION! In a barrier stage, each task must have the same number of barrier() calls, in all - * possible code branches. Otherwise, you may get the job hanging or a SparkException after - * timeout. Some examples of '''misuses''' are listed below: - * 1. Only call barrier() function on a subset of all the tasks in the same barrier stage, it - * shall lead to timeout of the function call. - * {{{ - * rdd.barrier().mapPartitions { iter => - * val context = BarrierTaskContext.get() - * if (context.partitionId() == 0) { - * // Do nothing. - * } else { - * context.barrier() - * } - * iter - * } - * }}} - * - * 2. Include barrier() function in a try-catch code block, this may lead to timeout of the - * second function call. - * {{{ - * rdd.barrier().mapPartitions { iter => - * val context = BarrierTaskContext.get() - * try { - * // Do something that might throw an Exception. - * doSomething() - * context.barrier() - * } catch { - * case e: Exception => logWarning("...", e) - * } - * context.barrier() - * iter - * } - * }}} - */ - @Experimental - @Since("2.4.0") - def barrier(): Unit = { + private def getRequestToSync( + numTasks: Int, + stageId: Int, + stageAttemptNumber: Int, + taskAttemptId: Long, + barrierEpoch: Int, + partitionId: Int, + requestMethod: RequestMethod.Value, + allGatherMessage: String + ): RequestToSync = { + requestMethod match { + case RequestMethod.BARRIER => + BarrierRequestToSync(numTasks, stageId, stageAttemptNumber, taskAttemptId, + barrierEpoch, partitionId, requestMethod) + case RequestMethod.ALL_GATHER => + AllGatherRequestToSync(numTasks, stageId, stageAttemptNumber, taskAttemptId, + barrierEpoch, partitionId, requestMethod, allGatherMessage) + } + } + + private def runBarrier( + requestMethod: RequestMethod.Value, + allGatherMessage: String = "" + ): String = { + logInfo(s"Task $taskAttemptId from Stage $stageId(Attempt $stageAttemptNumber) has entered " + s"the global sync, current barrier epoch is $barrierEpoch.") logTrace("Current callSite: " + Utils.getCallSite()) @@ -118,10 +108,12 @@ class BarrierTaskContext private[spark] ( // Log the update of global sync every 60 seconds. timer.schedule(timerTask, 60000, 60000) + var json: String = "" + try { - val abortableRpcFuture = barrierCoordinator.askAbortable[Unit]( - message = RequestToSync(numTasks, stageId, stageAttemptNumber, taskAttemptId, - barrierEpoch), + val abortableRpcFuture = barrierCoordinator.askAbortable[String]( + message = getRequestToSync(numTasks, stageId, stageAttemptNumber, + taskAttemptId, barrierEpoch, partitionId, requestMethod, allGatherMessage), // Set a fixed timeout for RPC here, so users shall get a SparkException thrown by // BarrierCoordinator on timeout, instead of RPCTimeoutException from the RPC framework. timeout = new RpcTimeout(365.days, "barrierTimeout")) @@ -133,7 +125,7 @@ class BarrierTaskContext private[spark] ( while (!abortableRpcFuture.toFuture.isCompleted) { // wait RPC future for at most 1 second try { - ThreadUtils.awaitResult(abortableRpcFuture.toFuture, 1.second) + json = ThreadUtils.awaitResult(abortableRpcFuture.toFuture, 1.second) } catch { case _: TimeoutException | _: InterruptedException => // If `TimeoutException` thrown, waiting RPC future reach 1 second. @@ -163,6 +155,73 @@ class BarrierTaskContext private[spark] ( timerTask.cancel() timer.purge() } + json + } + + /** + * :: Experimental :: + * Sets a global barrier and waits until all tasks in this stage hit this barrier. Similar to + * MPI_Barrier function in MPI, the barrier() function call blocks until all tasks in the same + * stage have reached this routine. + * + * CAUTION! In a barrier stage, each task must have the same number of barrier() calls, in all + * possible code branches. Otherwise, you may get the job hanging or a SparkException after + * timeout. Some examples of '''misuses''' are listed below: + * 1. Only call barrier() function on a subset of all the tasks in the same barrier stage, it + * shall lead to timeout of the function call. + * {{{ + * rdd.barrier().mapPartitions { iter => + * val context = BarrierTaskContext.get() + * if (context.partitionId() == 0) { + * // Do nothing. + * } else { + * context.barrier() + * } + * iter + * } + * }}} + * + * 2. Include barrier() function in a try-catch code block, this may lead to timeout of the + * second function call. + * {{{ + * rdd.barrier().mapPartitions { iter => + * val context = BarrierTaskContext.get() + * try { + * // Do something that might throw an Exception. + * doSomething() + * context.barrier() + * } catch { + * case e: Exception => logWarning("...", e) + * } + * context.barrier() + * iter + * } + * }}} + */ + @Experimental + @Since("2.4.0") + def barrier(): Unit = { + runBarrier(RequestMethod.BARRIER) + () + } + + /** + * :: Experimental :: + * Blocks until all tasks in the same stage have reached this routine. Each task passes in + * a message and returns with a list of all the messages passed in by each of those tasks. + * + * CAUTION! The allGather method requires the same precautions as the barrier method + * + * The message is type String rather than Array[Byte] because it is more convenient for + * the user at the cost of worse performance. + */ + @Experimental + @Since("3.0.0") + def allGather(message: String): ArrayBuffer[String] = { + val json = runBarrier(RequestMethod.ALL_GATHER, message) + val jsonArray = parse(json) + implicit val formats = DefaultFormats + ArrayBuffer(jsonArray.extract[Array[String]]: _*) } /** diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala index cb965cb180207..00bd0063c9e3a 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala @@ -37,24 +37,29 @@ private[spark] trait ExecutorAllocationClient { /** * Update the cluster manager on our scheduling needs. Three bits of information are included * to help it make decisions. - * @param numExecutors The total number of executors we'd like to have. The cluster manager - * shouldn't kill any running executor to reach this number, but, - * if all existing executors were to die, this is the number of executors - * we'd want to be allocated. - * @param localityAwareTasks The number of tasks in all active stages that have a locality - * preferences. This includes running, pending, and completed tasks. - * @param hostToLocalTaskCount A map of hosts to the number of tasks from all active stages - * that would like to like to run on that host. - * This includes running, pending, and completed tasks. + * + * @param resourceProfileIdToNumExecutors The total number of executors we'd like to have per + * ResourceProfile id. The cluster manager shouldn't kill + * any running executor to reach this number, but, if all + * existing executors were to die, this is the number + * of executors we'd want to be allocated. + * @param numLocalityAwareTasksPerResourceProfileId The number of tasks in all active stages that + * have a locality preferences per + * ResourceProfile id. This includes running, + * pending, and completed tasks. + * @param hostToLocalTaskCount A map of ResourceProfile id to a map of hosts to the number of + * tasks from all active stages that would like to like to run on + * that host. This includes running, pending, and completed tasks. * @return whether the request is acknowledged by the cluster manager. */ private[spark] def requestTotalExecutors( - numExecutors: Int, - localityAwareTasks: Int, - hostToLocalTaskCount: Map[String, Int]): Boolean + resourceProfileIdToNumExecutors: Map[Int, Int], + numLocalityAwareTasksPerResourceProfileId: Map[Int, Int], + hostToLocalTaskCount: Map[Int, Map[String, Int]]): Boolean /** - * Request an additional number of executors from the cluster manager. + * Request an additional number of executors from the cluster manager for the default + * ResourceProfile. * @return whether the request is acknowledged by the cluster manager. */ def requestExecutors(numAdditionalExecutors: Int): Boolean diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 677386cc7a572..5cb3160711a90 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -29,6 +29,8 @@ import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Tests.TEST_SCHEDULE_INTERVAL import org.apache.spark.metrics.source.Source +import org.apache.spark.resource.ResourceProfile.UNKNOWN_RESOURCE_PROFILE_ID +import org.apache.spark.resource.ResourceProfileManager import org.apache.spark.scheduler._ import org.apache.spark.scheduler.dynalloc.ExecutorMonitor import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils} @@ -36,9 +38,9 @@ import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils} /** * An agent that dynamically allocates and removes executors based on the workload. * - * The ExecutorAllocationManager maintains a moving target number of executors which is periodically - * synced to the cluster manager. The target starts at a configured initial value and changes with - * the number of pending and running tasks. + * The ExecutorAllocationManager maintains a moving target number of executors, for each + * ResourceProfile, which is periodically synced to the cluster manager. The target starts + * at a configured initial value and changes with the number of pending and running tasks. * * Decreasing the target number of executors happens when the current target is more than needed to * handle the current load. The target number of executors is always truncated to the number of @@ -57,14 +59,18 @@ import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils} * quickly over time in case the maximum number of executors is very high. Otherwise, it will take * a long time to ramp up under heavy workloads. * - * The remove policy is simpler: If an executor has been idle for K seconds, meaning it has not - * been scheduled to run any tasks, then it is removed. Note that an executor caching any data + * The remove policy is simpler and is applied on each ResourceProfile separately. If an executor + * for that ResourceProfile has been idle for K seconds and the number of executors is more + * then what is needed for that ResourceProfile, meaning there are not enough tasks that could use + * the executor, then it is removed. Note that an executor caching any data * blocks will be removed if it has been idle for more than L seconds. * * There is no retry logic in either case because we make the assumption that the cluster manager * will eventually fulfill all requests it receives asynchronously. * - * The relevant Spark properties include the following: + * The relevant Spark properties are below. Each of these properties applies separately to + * every ResourceProfile. So if you set a minimum number of executors, that is a minimum + * for each ResourceProfile. * * spark.dynamicAllocation.enabled - Whether this feature is enabled * spark.dynamicAllocation.minExecutors - Lower bound on the number of executors @@ -95,7 +101,8 @@ private[spark] class ExecutorAllocationManager( listenerBus: LiveListenerBus, conf: SparkConf, cleaner: Option[ContextCleaner] = None, - clock: Clock = new SystemClock()) + clock: Clock = new SystemClock(), + resourceProfileManager: ResourceProfileManager) extends Logging { allocationManager => @@ -117,23 +124,23 @@ private[spark] class ExecutorAllocationManager( // During testing, the methods to actually kill and add executors are mocked out private val testing = conf.get(DYN_ALLOCATION_TESTING) - // TODO: The default value of 1 for spark.executor.cores works right now because dynamic - // allocation is only supported for YARN and the default number of cores per executor in YARN is - // 1, but it might need to be attained differently for different cluster managers - private val tasksPerExecutorForFullParallelism = - conf.get(EXECUTOR_CORES) / conf.get(CPUS_PER_TASK) - private val executorAllocationRatio = conf.get(DYN_ALLOCATION_EXECUTOR_ALLOCATION_RATIO) + private val defaultProfileId = resourceProfileManager.defaultResourceProfile.id + validateSettings() - // Number of executors to add in the next round - private var numExecutorsToAdd = 1 + // Number of executors to add for each ResourceProfile in the next round + private val numExecutorsToAddPerResourceProfileId = new mutable.HashMap[Int, Int] + numExecutorsToAddPerResourceProfileId(defaultProfileId) = 1 // The desired number of executors at this moment in time. If all our executors were to die, this // is the number of executors we would immediately want from the cluster manager. - private var numExecutorsTarget = initialNumExecutors + // Note every profile will be allowed to have initial number, + // we may want to make this configurable per Profile in the future + private val numExecutorsTargetPerResourceProfileId = new mutable.HashMap[Int, Int] + numExecutorsTargetPerResourceProfileId(defaultProfileId) = initialNumExecutors // A timestamp of when an addition should be triggered, or NOT_SET if it is not set // This is set when pending tasks are added but not scheduled yet @@ -165,11 +172,12 @@ private[spark] class ExecutorAllocationManager( // (2) an executor idle timeout has elapsed. @volatile private var initializing: Boolean = true - // Number of locality aware tasks, used for executor placement. - private var localityAwareTasks = 0 + // Number of locality aware tasks for each ResourceProfile, used for executor placement. + private var numLocalityAwareTasksPerResourceProfileId = new mutable.HashMap[Int, Int] + numLocalityAwareTasksPerResourceProfileId(defaultProfileId) = 0 - // Host to possible task running on it, used for executor placement. - private var hostToLocalTaskCount: Map[String, Int] = Map.empty + // ResourceProfile id to Host to possible task running on it, used for executor placement. + private var rpIdToHostToLocalTaskCount: Map[Int, Map[String, Int]] = Map.empty /** * Verify that the settings specified through the config are valid. @@ -233,7 +241,14 @@ private[spark] class ExecutorAllocationManager( } executor.scheduleWithFixedDelay(scheduleTask, 0, intervalMillis, TimeUnit.MILLISECONDS) - client.requestTotalExecutors(numExecutorsTarget, localityAwareTasks, hostToLocalTaskCount) + // copy the maps inside synchonize to ensure not being modified + val (numExecutorsTarget, numLocalityAware) = synchronized { + val numTarget = numExecutorsTargetPerResourceProfileId.toMap + val numLocality = numLocalityAwareTasksPerResourceProfileId.toMap + (numTarget, numLocality) + } + + client.requestTotalExecutors(numExecutorsTarget, numLocalityAware, rpIdToHostToLocalTaskCount) } /** @@ -253,20 +268,28 @@ private[spark] class ExecutorAllocationManager( */ def reset(): Unit = synchronized { addTime = 0L - numExecutorsTarget = initialNumExecutors + numExecutorsTargetPerResourceProfileId.keys.foreach { rpId => + numExecutorsTargetPerResourceProfileId(rpId) = initialNumExecutors + } executorMonitor.reset() } /** - * The maximum number of executors we would need under the current load to satisfy all running - * and pending tasks, rounded up. + * The maximum number of executors, for the ResourceProfile id passed in, that we would need + * under the current load to satisfy all running and pending tasks, rounded up. */ - private def maxNumExecutorsNeeded(): Int = { - val numRunningOrPendingTasks = listener.totalPendingTasks + listener.totalRunningTasks + private def maxNumExecutorsNeededPerResourceProfile(rpId: Int): Int = { + val pending = listener.totalPendingTasksPerResourceProfile(rpId) + val pendingSpeculative = listener.pendingSpeculativeTasksPerResourceProfile(rpId) + val running = listener.totalRunningTasksPerResourceProfile(rpId) + val numRunningOrPendingTasks = pending + running + val rp = resourceProfileManager.resourceProfileFromId(rpId) + val tasksPerExecutor = rp.maxTasksPerExecutor(conf) + logDebug(s"max needed for rpId: $rpId numpending: $numRunningOrPendingTasks," + + s" tasksperexecutor: $tasksPerExecutor") val maxNeeded = math.ceil(numRunningOrPendingTasks * executorAllocationRatio / - tasksPerExecutorForFullParallelism).toInt - if (tasksPerExecutorForFullParallelism > 1 && maxNeeded == 1 && - listener.pendingSpeculativeTasks > 0) { + tasksPerExecutor).toInt + if (tasksPerExecutor > 1 && maxNeeded == 1 && pendingSpeculative > 0) { // If we have pending speculative tasks and only need a single executor, allocate one more // to satisfy the locality requirements of speculation maxNeeded + 1 @@ -275,8 +298,8 @@ private[spark] class ExecutorAllocationManager( } } - private def totalRunningTasks(): Int = synchronized { - listener.totalRunningTasks + private def totalRunningTasksPerResourceProfile(id: Int): Int = synchronized { + listener.totalRunningTasksPerResourceProfile(id) } /** @@ -302,7 +325,8 @@ private[spark] class ExecutorAllocationManager( } /** - * Updates our target number of executors and syncs the result with the cluster manager. + * Updates our target number of executors for each ResourceProfile and then syncs the result + * with the cluster manager. * * Check to see whether our existing allocation and the requests we've made previously exceed our * current needs. If so, truncate our target and let the cluster manager know so that it can @@ -314,130 +338,205 @@ private[spark] class ExecutorAllocationManager( * @return the delta in the target number of executors. */ private def updateAndSyncNumExecutorsTarget(now: Long): Int = synchronized { - val maxNeeded = maxNumExecutorsNeeded - if (initializing) { // Do not change our target while we are still initializing, // Otherwise the first job may have to ramp up unnecessarily 0 - } else if (maxNeeded < numExecutorsTarget) { - // The target number exceeds the number we actually need, so stop adding new - // executors and inform the cluster manager to cancel the extra pending requests - val oldNumExecutorsTarget = numExecutorsTarget - numExecutorsTarget = math.max(maxNeeded, minNumExecutors) - numExecutorsToAdd = 1 - - // If the new target has not changed, avoid sending a message to the cluster manager - if (numExecutorsTarget < oldNumExecutorsTarget) { - // We lower the target number of executors but don't actively kill any yet. Killing is - // controlled separately by an idle timeout. It's still helpful to reduce the target number - // in case an executor just happens to get lost (eg., bad hardware, or the cluster manager - // preempts it) -- in that case, there is no point in trying to immediately get a new - // executor, since we wouldn't even use it yet. - client.requestTotalExecutors(numExecutorsTarget, localityAwareTasks, hostToLocalTaskCount) - logDebug(s"Lowering target number of executors to $numExecutorsTarget (previously " + - s"$oldNumExecutorsTarget) because not all requested executors are actually needed") + } else { + val updatesNeeded = new mutable.HashMap[Int, ExecutorAllocationManager.TargetNumUpdates] + + // Update targets for all ResourceProfiles then do a single request to the cluster manager + numExecutorsTargetPerResourceProfileId.foreach { case (rpId, targetExecs) => + val maxNeeded = maxNumExecutorsNeededPerResourceProfile(rpId) + if (maxNeeded < targetExecs) { + // The target number exceeds the number we actually need, so stop adding new + // executors and inform the cluster manager to cancel the extra pending requests + + // We lower the target number of executors but don't actively kill any yet. Killing is + // controlled separately by an idle timeout. It's still helpful to reduce + // the target number in case an executor just happens to get lost (eg., bad hardware, + // or the cluster manager preempts it) -- in that case, there is no point in trying + // to immediately get a new executor, since we wouldn't even use it yet. + decrementExecutorsFromTarget(maxNeeded, rpId, updatesNeeded) + } else if (addTime != NOT_SET && now >= addTime) { + addExecutorsToTarget(maxNeeded, rpId, updatesNeeded) + } + } + doUpdateRequest(updatesNeeded.toMap, now) + } + } + + private def addExecutorsToTarget( + maxNeeded: Int, + rpId: Int, + updatesNeeded: mutable.HashMap[Int, ExecutorAllocationManager.TargetNumUpdates]): Int = { + updateTargetExecs(addExecutors, maxNeeded, rpId, updatesNeeded) + } + + private def decrementExecutorsFromTarget( + maxNeeded: Int, + rpId: Int, + updatesNeeded: mutable.HashMap[Int, ExecutorAllocationManager.TargetNumUpdates]): Int = { + updateTargetExecs(decrementExecutors, maxNeeded, rpId, updatesNeeded) + } + + private def updateTargetExecs( + updateTargetFn: (Int, Int) => Int, + maxNeeded: Int, + rpId: Int, + updatesNeeded: mutable.HashMap[Int, ExecutorAllocationManager.TargetNumUpdates]): Int = { + val oldNumExecutorsTarget = numExecutorsTargetPerResourceProfileId(rpId) + // update the target number (add or remove) + val delta = updateTargetFn(maxNeeded, rpId) + if (delta != 0) { + updatesNeeded(rpId) = ExecutorAllocationManager.TargetNumUpdates(delta, oldNumExecutorsTarget) + } + delta + } + + private def doUpdateRequest( + updates: Map[Int, ExecutorAllocationManager.TargetNumUpdates], + now: Long): Int = { + // Only call cluster manager if target has changed. + if (updates.size > 0) { + val requestAcknowledged = try { + logDebug("requesting updates: " + updates) + testing || + client.requestTotalExecutors( + numExecutorsTargetPerResourceProfileId.toMap, + numLocalityAwareTasksPerResourceProfileId.toMap, + rpIdToHostToLocalTaskCount) + } catch { + case NonFatal(e) => + // Use INFO level so the error it doesn't show up by default in shells. + // Errors here are more commonly caused by YARN AM restarts, which is a recoverable + // issue, and generate a lot of noisy output. + logInfo("Error reaching cluster manager.", e) + false + } + if (requestAcknowledged) { + // have to go through all resource profiles that changed + var totalDelta = 0 + updates.foreach { case (rpId, targetNum) => + val delta = targetNum.delta + totalDelta += delta + if (delta > 0) { + val executorsString = "executor" + { if (delta > 1) "s" else "" } + logInfo(s"Requesting $delta new $executorsString because tasks are backlogged " + + s"(new desired total will be ${numExecutorsTargetPerResourceProfileId(rpId)} " + + s"for resource profile id: ${rpId})") + numExecutorsToAddPerResourceProfileId(rpId) = + if (delta == numExecutorsToAddPerResourceProfileId(rpId)) { + numExecutorsToAddPerResourceProfileId(rpId) * 2 + } else { + 1 + } + logDebug(s"Starting timer to add more executors (to " + + s"expire in $sustainedSchedulerBacklogTimeoutS seconds)") + addTime = now + TimeUnit.SECONDS.toNanos(sustainedSchedulerBacklogTimeoutS) + } else { + logDebug(s"Lowering target number of executors to" + + s" ${numExecutorsTargetPerResourceProfileId(rpId)} (previously " + + s"$targetNum.oldNumExecutorsTarget for resource profile id: ${rpId}) " + + "because not all requested executors " + + "are actually needed") + } + } + totalDelta + } else { + // request was for all profiles so we have to go through all to reset to old num + updates.foreach { case (rpId, targetNum) => + logWarning("Unable to reach the cluster manager to request more executors!") + numExecutorsTargetPerResourceProfileId(rpId) = targetNum.oldNumExecutorsTarget + } + 0 } - numExecutorsTarget - oldNumExecutorsTarget - } else if (addTime != NOT_SET && now >= addTime) { - val delta = addExecutors(maxNeeded) - logDebug(s"Starting timer to add more executors (to " + - s"expire in $sustainedSchedulerBacklogTimeoutS seconds)") - addTime = now + TimeUnit.SECONDS.toNanos(sustainedSchedulerBacklogTimeoutS) - delta } else { + logDebug("No change in number of executors") 0 } } + private def decrementExecutors(maxNeeded: Int, rpId: Int): Int = { + val oldNumExecutorsTarget = numExecutorsTargetPerResourceProfileId(rpId) + numExecutorsTargetPerResourceProfileId(rpId) = math.max(maxNeeded, minNumExecutors) + numExecutorsToAddPerResourceProfileId(rpId) = 1 + numExecutorsTargetPerResourceProfileId(rpId) - oldNumExecutorsTarget + } + /** - * Request a number of executors from the cluster manager. + * Update the target number of executors and figure out how many to add. * If the cap on the number of executors is reached, give up and reset the * number of executors to add next round instead of continuing to double it. * * @param maxNumExecutorsNeeded the maximum number of executors all currently running or pending * tasks could fill + * @param rpId the ResourceProfile id of the executors * @return the number of additional executors actually requested. */ - private def addExecutors(maxNumExecutorsNeeded: Int): Int = { + private def addExecutors(maxNumExecutorsNeeded: Int, rpId: Int): Int = { + val oldNumExecutorsTarget = numExecutorsTargetPerResourceProfileId(rpId) // Do not request more executors if it would put our target over the upper bound - if (numExecutorsTarget >= maxNumExecutors) { - logDebug(s"Not adding executors because our current target total " + - s"is already $numExecutorsTarget (limit $maxNumExecutors)") - numExecutorsToAdd = 1 + // this is doing a max check per ResourceProfile + if (oldNumExecutorsTarget >= maxNumExecutors) { + logDebug("Not adding executors because our current target total " + + s"is already ${oldNumExecutorsTarget} (limit $maxNumExecutors)") + numExecutorsToAddPerResourceProfileId(rpId) = 1 return 0 } - - val oldNumExecutorsTarget = numExecutorsTarget // There's no point in wasting time ramping up to the number of executors we already have, so // make sure our target is at least as much as our current allocation: - numExecutorsTarget = math.max(numExecutorsTarget, executorMonitor.executorCount) + var numExecutorsTarget = math.max(numExecutorsTargetPerResourceProfileId(rpId), + executorMonitor.executorCountWithResourceProfile(rpId)) // Boost our target with the number to add for this round: - numExecutorsTarget += numExecutorsToAdd + numExecutorsTarget += numExecutorsToAddPerResourceProfileId(rpId) // Ensure that our target doesn't exceed what we need at the present moment: numExecutorsTarget = math.min(numExecutorsTarget, maxNumExecutorsNeeded) // Ensure that our target fits within configured bounds: numExecutorsTarget = math.max(math.min(numExecutorsTarget, maxNumExecutors), minNumExecutors) - val delta = numExecutorsTarget - oldNumExecutorsTarget + numExecutorsTargetPerResourceProfileId(rpId) = numExecutorsTarget // If our target has not changed, do not send a message // to the cluster manager and reset our exponential growth if (delta == 0) { - numExecutorsToAdd = 1 - return 0 - } - - val addRequestAcknowledged = try { - testing || - client.requestTotalExecutors(numExecutorsTarget, localityAwareTasks, hostToLocalTaskCount) - } catch { - case NonFatal(e) => - // Use INFO level so the error it doesn't show up by default in shells. Errors here are more - // commonly caused by YARN AM restarts, which is a recoverable issue, and generate a lot of - // noisy output. - logInfo("Error reaching cluster manager.", e) - false - } - if (addRequestAcknowledged) { - val executorsString = "executor" + { if (delta > 1) "s" else "" } - logInfo(s"Requesting $delta new $executorsString because tasks are backlogged" + - s" (new desired total will be $numExecutorsTarget)") - numExecutorsToAdd = if (delta == numExecutorsToAdd) { - numExecutorsToAdd * 2 - } else { - 1 - } - delta - } else { - logWarning( - s"Unable to reach the cluster manager to request $numExecutorsTarget total executors!") - numExecutorsTarget = oldNumExecutorsTarget - 0 + numExecutorsToAddPerResourceProfileId(rpId) = 1 } + delta } /** * Request the cluster manager to remove the given executors. * Returns the list of executors which are removed. */ - private def removeExecutors(executors: Seq[String]): Seq[String] = synchronized { + private def removeExecutors(executors: Seq[(String, Int)]): Seq[String] = synchronized { val executorIdsToBeRemoved = new ArrayBuffer[String] - logDebug(s"Request to remove executorIds: ${executors.mkString(", ")}") - val numExistingExecutors = executorMonitor.executorCount - executorMonitor.pendingRemovalCount - - var newExecutorTotal = numExistingExecutors - executors.foreach { executorIdToBeRemoved => - if (newExecutorTotal - 1 < minNumExecutors) { - logDebug(s"Not removing idle executor $executorIdToBeRemoved because there are only " + - s"$newExecutorTotal executor(s) left (minimum number of executor limit $minNumExecutors)") - } else if (newExecutorTotal - 1 < numExecutorsTarget) { - logDebug(s"Not removing idle executor $executorIdToBeRemoved because there are only " + - s"$newExecutorTotal executor(s) left (number of executor target $numExecutorsTarget)") + val numExecutorsTotalPerRpId = mutable.Map[Int, Int]() + executors.foreach { case (executorIdToBeRemoved, rpId) => + if (rpId == UNKNOWN_RESOURCE_PROFILE_ID) { + if (testing) { + throw new SparkException("ResourceProfile Id was UNKNOWN, this is not expected") + } + logWarning(s"Not removing executor $executorIdsToBeRemoved because the " + + "ResourceProfile was UNKNOWN!") } else { - executorIdsToBeRemoved += executorIdToBeRemoved - newExecutorTotal -= 1 + // get the running total as we remove or initialize it to the count - pendingRemoval + val newExecutorTotal = numExecutorsTotalPerRpId.getOrElseUpdate(rpId, + (executorMonitor.executorCountWithResourceProfile(rpId) - + executorMonitor.pendingRemovalCountPerResourceProfileId(rpId))) + if (newExecutorTotal - 1 < minNumExecutors) { + logDebug(s"Not removing idle executor $executorIdToBeRemoved because there " + + s"are only $newExecutorTotal executor(s) left (minimum number of executor limit " + + s"$minNumExecutors)") + } else if (newExecutorTotal - 1 < numExecutorsTargetPerResourceProfileId(rpId)) { + logDebug(s"Not removing idle executor $executorIdToBeRemoved because there " + + s"are only $newExecutorTotal executor(s) left (number of executor " + + s"target ${numExecutorsTargetPerResourceProfileId(rpId)})") + } else { + executorIdsToBeRemoved += executorIdToBeRemoved + numExecutorsTotalPerRpId(rpId) -= 1 + } } } @@ -457,14 +556,15 @@ private[spark] class ExecutorAllocationManager( // [SPARK-21834] killExecutors api reduces the target number of executors. // So we need to update the target with desired value. - client.requestTotalExecutors(numExecutorsTarget, localityAwareTasks, hostToLocalTaskCount) + client.requestTotalExecutors( + numExecutorsTargetPerResourceProfileId.toMap, + numLocalityAwareTasksPerResourceProfileId.toMap, + rpIdToHostToLocalTaskCount) + // reset the newExecutorTotal to the existing number of executors - newExecutorTotal = numExistingExecutors if (testing || executorsRemoved.nonEmpty) { - newExecutorTotal -= executorsRemoved.size executorMonitor.executorsKilled(executorsRemoved) - logInfo(s"Executors ${executorsRemoved.mkString(",")} removed due to idle timeout." + - s"(new desired total will be $newExecutorTotal)") + logInfo(s"Executors ${executorsRemoved.mkString(",")} removed due to idle timeout.") executorsRemoved } else { logWarning(s"Unable to reach the cluster manager to kill executor/s " + @@ -493,7 +593,7 @@ private[spark] class ExecutorAllocationManager( private def onSchedulerQueueEmpty(): Unit = synchronized { logDebug("Clearing timer to add executors because there are no more pending tasks") addTime = NOT_SET - numExecutorsToAdd = 1 + numExecutorsToAddPerResourceProfileId.transform { case (_, _) => 1 } } private case class StageAttempt(stageId: Int, stageAttemptId: Int) { @@ -519,12 +619,16 @@ private[spark] class ExecutorAllocationManager( private val stageAttemptToSpeculativeTaskIndices = new mutable.HashMap[StageAttempt, mutable.HashSet[Int]] + private val resourceProfileIdToStageAttempt = + new mutable.HashMap[Int, mutable.Set[StageAttempt]] + // stageAttempt to tuple (the number of task with locality preferences, a map where each pair - // is a node and the number of tasks that would like to be scheduled on that node) map, + // is a node and the number of tasks that would like to be scheduled on that node, and + // the resource profile id) map, // maintain the executor placement hints for each stageAttempt used by resource framework // to better place the executors. private val stageAttemptToExecutorPlacementHints = - new mutable.HashMap[StageAttempt, (Int, Map[String, Int])] + new mutable.HashMap[StageAttempt, (Int, Map[String, Int], Int)] override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = { initializing = false @@ -535,6 +639,13 @@ private[spark] class ExecutorAllocationManager( allocationManager.synchronized { stageAttemptToNumTasks(stageAttempt) = numTasks allocationManager.onSchedulerBacklogged() + // need to keep stage task requirements to ask for the right containers + val profId = stageSubmitted.stageInfo.resourceProfileId + logDebug(s"Stage resource profile id is: $profId with numTasks: $numTasks") + resourceProfileIdToStageAttempt.getOrElseUpdate( + profId, new mutable.HashSet[StageAttempt]) += stageAttempt + numExecutorsToAddPerResourceProfileId.getOrElseUpdate(profId, 1) + numExecutorsTargetPerResourceProfileId.getOrElseUpdate(profId, initialNumExecutors) // Compute the number of tasks requested by the stage on each host var numTasksPending = 0 @@ -549,7 +660,7 @@ private[spark] class ExecutorAllocationManager( } } stageAttemptToExecutorPlacementHints.put(stageAttempt, - (numTasksPending, hostToLocalTaskCountPerStage.toMap)) + (numTasksPending, hostToLocalTaskCountPerStage.toMap, profId)) // Update the executor placement hints updateExecutorPlacementHints() @@ -561,7 +672,7 @@ private[spark] class ExecutorAllocationManager( val stageAttemptId = stageCompleted.stageInfo.attemptNumber() val stageAttempt = StageAttempt(stageId, stageAttemptId) allocationManager.synchronized { - // do NOT remove stageAttempt from stageAttemptToNumRunningTasks, + // do NOT remove stageAttempt from stageAttemptToNumRunningTask // because the attempt may still have running tasks, // even after another attempt for the stage is submitted. stageAttemptToNumTasks -= stageAttempt @@ -597,7 +708,7 @@ private[spark] class ExecutorAllocationManager( stageAttemptToTaskIndices.getOrElseUpdate(stageAttempt, new mutable.HashSet[Int]) += taskIndex } - if (totalPendingTasks() == 0) { + if (!hasPendingTasks) { allocationManager.onSchedulerQueueEmpty() } } @@ -613,9 +724,22 @@ private[spark] class ExecutorAllocationManager( stageAttemptToNumRunningTask(stageAttempt) -= 1 if (stageAttemptToNumRunningTask(stageAttempt) == 0) { stageAttemptToNumRunningTask -= stageAttempt + if (!stageAttemptToNumTasks.contains(stageAttempt)) { + val rpForStage = resourceProfileIdToStageAttempt.filter { case (k, v) => + v.contains(stageAttempt) + }.keys + if (rpForStage.size == 1) { + // be careful about the removal from here due to late tasks, make sure stage is + // really complete and no tasks left + resourceProfileIdToStageAttempt(rpForStage.head) -= stageAttempt + } else { + logWarning(s"Should have exactly one resource profile for stage $stageAttempt," + + s" but have $rpForStage") + } + } + } } - if (taskEnd.taskInfo.speculative) { stageAttemptToSpeculativeTaskIndices.get(stageAttempt).foreach {_.remove{taskIndex}} stageAttemptToNumSpeculativeTasks(stageAttempt) -= 1 @@ -624,7 +748,7 @@ private[spark] class ExecutorAllocationManager( taskEnd.reason match { case Success | _: TaskKilled => case _ => - if (totalPendingTasks() == 0) { + if (!hasPendingTasks) { // If the task failed (not intentionally killed), we expect it to be resubmitted // later. To ensure we have enough resources to run the resubmitted task, we need to // mark the scheduler as backlogged again if it's not already marked as such @@ -661,20 +785,46 @@ private[spark] class ExecutorAllocationManager( * * Note: This is not thread-safe without the caller owning the `allocationManager` lock. */ - def pendingTasks(): Int = { - stageAttemptToNumTasks.map { case (stageAttempt, numTasks) => - numTasks - stageAttemptToTaskIndices.get(stageAttempt).map(_.size).getOrElse(0) - }.sum + def pendingTasksPerResourceProfile(rpId: Int): Int = { + val attempts = resourceProfileIdToStageAttempt.getOrElse(rpId, Set.empty).toSeq + attempts.map(attempt => getPendingTaskSum(attempt)).sum } - def pendingSpeculativeTasks(): Int = { - stageAttemptToNumSpeculativeTasks.map { case (stageAttempt, numTasks) => - numTasks - stageAttemptToSpeculativeTaskIndices.get(stageAttempt).map(_.size).getOrElse(0) - }.sum + def hasPendingRegularTasks: Boolean = { + val attemptSets = resourceProfileIdToStageAttempt.values + attemptSets.exists(attempts => attempts.exists(getPendingTaskSum(_) > 0)) + } + + private def getPendingTaskSum(attempt: StageAttempt): Int = { + val numTotalTasks = stageAttemptToNumTasks.getOrElse(attempt, 0) + val numRunning = stageAttemptToTaskIndices.get(attempt).map(_.size).getOrElse(0) + numTotalTasks - numRunning } - def totalPendingTasks(): Int = { - pendingTasks + pendingSpeculativeTasks + def pendingSpeculativeTasksPerResourceProfile(rp: Int): Int = { + val attempts = resourceProfileIdToStageAttempt.getOrElse(rp, Set.empty).toSeq + attempts.map(attempt => getPendingSpeculativeTaskSum(attempt)).sum + } + + def hasPendingSpeculativeTasks: Boolean = { + val attemptSets = resourceProfileIdToStageAttempt.values + attemptSets.exists { attempts => + attempts.exists(getPendingSpeculativeTaskSum(_) > 0) + } + } + + private def getPendingSpeculativeTaskSum(attempt: StageAttempt): Int = { + val numTotalTasks = stageAttemptToNumSpeculativeTasks.getOrElse(attempt, 0) + val numRunning = stageAttemptToSpeculativeTaskIndices.get(attempt).map(_.size).getOrElse(0) + numTotalTasks - numRunning + } + + def hasPendingTasks: Boolean = { + hasPendingSpeculativeTasks || hasPendingRegularTasks + } + + def totalPendingTasksPerResourceProfile(rp: Int): Int = { + pendingTasksPerResourceProfile(rp) + pendingSpeculativeTasksPerResourceProfile(rp) } /** @@ -685,6 +835,14 @@ private[spark] class ExecutorAllocationManager( stageAttemptToNumRunningTask.values.sum } + def totalRunningTasksPerResourceProfile(rp: Int): Int = { + val attempts = resourceProfileIdToStageAttempt.getOrElse(rp, Set.empty).toSeq + // attempts is a Set, change to Seq so we keep all values + attempts.map { attempt => + stageAttemptToNumRunningTask.getOrElseUpdate(attempt, 0) + }.sum + } + /** * Update the Executor placement hints (the number of tasks with locality preferences, * a map where each pair is a node and the number of tasks that would like to be scheduled @@ -694,18 +852,27 @@ private[spark] class ExecutorAllocationManager( * granularity within stages. */ def updateExecutorPlacementHints(): Unit = { - var localityAwareTasks = 0 - val localityToCount = new mutable.HashMap[String, Int]() - stageAttemptToExecutorPlacementHints.values.foreach { case (numTasksPending, localities) => - localityAwareTasks += numTasksPending - localities.foreach { case (hostname, count) => - val updatedCount = localityToCount.getOrElse(hostname, 0) + count - localityToCount(hostname) = updatedCount - } + val localityAwareTasksPerResourceProfileId = new mutable.HashMap[Int, Int] + + // ResourceProfile id => map[host, count] + val rplocalityToCount = new mutable.HashMap[Int, mutable.HashMap[String, Int]]() + stageAttemptToExecutorPlacementHints.values.foreach { + case (numTasksPending, localities, rpId) => + val rpNumPending = + localityAwareTasksPerResourceProfileId.getOrElse(rpId, 0) + localityAwareTasksPerResourceProfileId(rpId) = rpNumPending + numTasksPending + localities.foreach { case (hostname, count) => + val rpBasedHostToCount = + rplocalityToCount.getOrElseUpdate(rpId, new mutable.HashMap[String, Int]) + val newUpdated = rpBasedHostToCount.getOrElse(hostname, 0) + count + rpBasedHostToCount(hostname) = newUpdated + } } - allocationManager.localityAwareTasks = localityAwareTasks - allocationManager.hostToLocalTaskCount = localityToCount.toMap + allocationManager.numLocalityAwareTasksPerResourceProfileId = + localityAwareTasksPerResourceProfileId + allocationManager.rpIdToHostToLocalTaskCount = + rplocalityToCount.map { case (k, v) => (k, v.toMap)}.toMap } } @@ -726,14 +893,22 @@ private[spark] class ExecutorAllocationManager( }) } - registerGauge("numberExecutorsToAdd", numExecutorsToAdd, 0) + // The metrics are going to return the sum for all the different ResourceProfiles. + registerGauge("numberExecutorsToAdd", + numExecutorsToAddPerResourceProfileId.values.sum, 0) registerGauge("numberExecutorsPendingToRemove", executorMonitor.pendingRemovalCount, 0) registerGauge("numberAllExecutors", executorMonitor.executorCount, 0) - registerGauge("numberTargetExecutors", numExecutorsTarget, 0) - registerGauge("numberMaxNeededExecutors", maxNumExecutorsNeeded(), 0) + registerGauge("numberTargetExecutors", + numExecutorsTargetPerResourceProfileId.values.sum, 0) + registerGauge("numberMaxNeededExecutors", numExecutorsTargetPerResourceProfileId.keys + .map(maxNumExecutorsNeededPerResourceProfile(_)).sum, 0) } } private object ExecutorAllocationManager { val NOT_SET = Long.MaxValue + + // helper case class for requesting executors, here to be visible for testing + private[spark] case class TargetNumUpdates(delta: Int, oldNumExecutorsTarget: Int) + } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 91188d58f4201..f377f13d30ec2 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -25,6 +25,7 @@ import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger, AtomicReferenc import scala.collection.JavaConverters._ import scala.collection.Map +import scala.collection.immutable import scala.collection.mutable.HashMap import scala.language.implicitConversions import scala.reflect.{classTag, ClassTag} @@ -53,7 +54,7 @@ import org.apache.spark.io.CompressionCodec import org.apache.spark.metrics.source.JVMCPUSource import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} import org.apache.spark.rdd._ -import org.apache.spark.resource.{ResourceID, ResourceInformation} +import org.apache.spark.resource._ import org.apache.spark.resource.ResourceUtils._ import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.scheduler._ @@ -219,9 +220,10 @@ class SparkContext(config: SparkConf) extends Logging { private var _shutdownHookRef: AnyRef = _ private var _statusStore: AppStatusStore = _ private var _heartbeater: Heartbeater = _ - private var _resources: scala.collection.immutable.Map[String, ResourceInformation] = _ + private var _resources: immutable.Map[String, ResourceInformation] = _ private var _shuffleDriverComponents: ShuffleDriverComponents = _ private var _plugins: Option[PluginContainer] = None + private var _resourceProfileManager: ResourceProfileManager = _ /* ------------------------------------------------------------------------------------- * | Accessors and public fields. These provide access to the internal state of the | @@ -343,6 +345,8 @@ class SparkContext(config: SparkConf) extends Logging { private[spark] def executorAllocationManager: Option[ExecutorAllocationManager] = _executorAllocationManager + private[spark] def resourceProfileManager: ResourceProfileManager = _resourceProfileManager + private[spark] def cleaner: Option[ContextCleaner] = _cleaner private[spark] var checkpointDir: Option[String] = None @@ -451,6 +455,7 @@ class SparkContext(config: SparkConf) extends Logging { } _listenerBus = new LiveListenerBus(_conf) + _resourceProfileManager = new ResourceProfileManager(_conf) // Initialize the app status store and listener before SparkEnv is created so that it gets // all events. @@ -611,7 +616,7 @@ class SparkContext(config: SparkConf) extends Logging { case b: ExecutorAllocationClient => Some(new ExecutorAllocationManager( schedulerBackend.asInstanceOf[ExecutorAllocationClient], listenerBus, _conf, - cleaner = cleaner)) + cleaner = cleaner, resourceProfileManager = resourceProfileManager)) case _ => None } @@ -1622,7 +1627,7 @@ class SparkContext(config: SparkConf) extends Logging { /** * Update the cluster manager on our scheduling needs. Three bits of information are included - * to help it make decisions. + * to help it make decisions. This applies to the default ResourceProfile. * @param numExecutors The total number of executors we'd like to have. The cluster manager * shouldn't kill any running executor to reach this number, but, * if all existing executors were to die, this is the number of executors @@ -1638,11 +1643,16 @@ class SparkContext(config: SparkConf) extends Logging { def requestTotalExecutors( numExecutors: Int, localityAwareTasks: Int, - hostToLocalTaskCount: scala.collection.immutable.Map[String, Int] + hostToLocalTaskCount: immutable.Map[String, Int] ): Boolean = { schedulerBackend match { case b: ExecutorAllocationClient => - b.requestTotalExecutors(numExecutors, localityAwareTasks, hostToLocalTaskCount) + // this is being applied to the default resource profile, would need to add api to support + // others + val defaultProfId = resourceProfileManager.defaultResourceProfile.id + b.requestTotalExecutors(immutable.Map(defaultProfId-> numExecutors), + immutable.Map(localityAwareTasks -> defaultProfId), + immutable.Map(defaultProfId -> hostToLocalTaskCount)) case _ => logWarning("Requesting executors is not supported by current scheduler.") false @@ -2036,6 +2046,7 @@ class SparkContext(config: SparkConf) extends Logging { // Clear this `InheritableThreadLocal`, or it will still be inherited in child threads even this // `SparkContext` is stopped. localProperties.remove() + ResourceProfile.clearDefaultProfile() // Unset YARN mode system env variable, to allow switching between cluster types. SparkContext.clearActiveContext() logInfo("Successfully stopped SparkContext") @@ -2771,109 +2782,34 @@ object SparkContext extends Logging { // When running locally, don't try to re-execute tasks on failure. val MAX_LOCAL_TASK_FAILURES = 1 - // Ensure that executor's resources satisfies one or more tasks requirement. - def checkResourcesPerTask(clusterMode: Boolean, executorCores: Option[Int]): Unit = { + // Ensure that default executor's resources satisfies one or more tasks requirement. + // This function is for cluster managers that don't set the executor cores config, for + // others its checked in ResourceProfile. + def checkResourcesPerTask(executorCores: Int): Unit = { val taskCores = sc.conf.get(CPUS_PER_TASK) - val execCores = if (clusterMode) { - executorCores.getOrElse(sc.conf.get(EXECUTOR_CORES)) - } else { - executorCores.get - } - // some cluster managers don't set the EXECUTOR_CORES config by default (standalone - // and mesos coarse grained), so we can't rely on that config for those. - val shouldCheckExecCores = executorCores.isDefined || sc.conf.contains(EXECUTOR_CORES) || - (master.equalsIgnoreCase("yarn") || master.startsWith("k8s")) - - // Number of cores per executor must meet at least one task requirement. - if (shouldCheckExecCores && execCores < taskCores) { - throw new SparkException(s"The number of cores per executor (=$execCores) has to be >= " + - s"the task config: ${CPUS_PER_TASK.key} = $taskCores when run on $master.") - } - - // Calculate the max slots each executor can provide based on resources available on each - // executor and resources required by each task. - val taskResourceRequirements = parseResourceRequirements(sc.conf, SPARK_TASK_PREFIX) - val executorResourcesAndAmounts = parseAllResourceRequests(sc.conf, SPARK_EXECUTOR_PREFIX) - .map(request => (request.id.resourceName, request.amount)).toMap - - var (numSlots, limitingResourceName) = if (shouldCheckExecCores) { - (execCores / taskCores, "CPU") - } else { - (-1, "") - } - - taskResourceRequirements.foreach { taskReq => - // Make sure the executor resources were specified through config. - val execAmount = executorResourcesAndAmounts.getOrElse(taskReq.resourceName, - throw new SparkException("The executor resource config: " + - new ResourceID(SPARK_EXECUTOR_PREFIX, taskReq.resourceName).amountConf + - " needs to be specified since a task requirement config: " + - new ResourceID(SPARK_TASK_PREFIX, taskReq.resourceName).amountConf + - " was specified") - ) - // Make sure the executor resources are large enough to launch at least one task. - if (execAmount < taskReq.amount) { - throw new SparkException("The executor resource config: " + - new ResourceID(SPARK_EXECUTOR_PREFIX, taskReq.resourceName).amountConf + - s" = $execAmount has to be >= the requested amount in task resource config: " + - new ResourceID(SPARK_TASK_PREFIX, taskReq.resourceName).amountConf + - s" = ${taskReq.amount}") - } - // Compare and update the max slots each executor can provide. - // If the configured amount per task was < 1.0, a task is subdividing - // executor resources. If the amount per task was > 1.0, the task wants - // multiple executor resources. - val resourceNumSlots = Math.floor(execAmount * taskReq.numParts / taskReq.amount).toInt - if (resourceNumSlots < numSlots) { - if (shouldCheckExecCores) { - throw new IllegalArgumentException("The number of slots on an executor has to be " + - "limited by the number of cores, otherwise you waste resources and " + - "dynamic allocation doesn't work properly. Your configuration has " + - s"core/task cpu slots = ${numSlots} and " + - s"${taskReq.resourceName} = ${resourceNumSlots}. " + - "Please adjust your configuration so that all resources require same number " + - "of executor slots.") - } - numSlots = resourceNumSlots - limitingResourceName = taskReq.resourceName - } - } - if(!shouldCheckExecCores && Utils.isDynamicAllocationEnabled(sc.conf)) { - // if we can't rely on the executor cores config throw a warning for user - logWarning("Please ensure that the number of slots available on your " + - "executors is limited by the number of cores to task cpus and not another " + - "custom resource. If cores is not the limiting resource then dynamic " + - "allocation will not work properly!") - } - // warn if we would waste any resources due to another resource limiting the number of - // slots on an executor - taskResourceRequirements.foreach { taskReq => - val execAmount = executorResourcesAndAmounts(taskReq.resourceName) - if ((numSlots * taskReq.amount / taskReq.numParts) < execAmount) { - val taskReqStr = if (taskReq.numParts > 1) { - s"${taskReq.amount}/${taskReq.numParts}" - } else { - s"${taskReq.amount}" - } - val resourceNumSlots = Math.floor(execAmount * taskReq.numParts / taskReq.amount).toInt - val message = s"The configuration of resource: ${taskReq.resourceName} " + - s"(exec = ${execAmount}, task = ${taskReqStr}, " + - s"runnable tasks = ${resourceNumSlots}) will " + - s"result in wasted resources due to resource ${limitingResourceName} limiting the " + - s"number of runnable tasks per executor to: ${numSlots}. Please adjust " + - s"your configuration." - if (Utils.isTesting) { - throw new SparkException(message) - } else { - logWarning(message) - } - } + validateTaskCpusLargeEnough(executorCores, taskCores) + val defaultProf = sc.resourceProfileManager.defaultResourceProfile + // TODO - this is temporary until all of stage level scheduling feature is integrated, + // fail if any other resource limiting due to dynamic allocation and scheduler using + // slots based on cores + val cpuSlots = executorCores/taskCores + val limitingResource = defaultProf.limitingResource(sc.conf) + if (limitingResource.nonEmpty && !limitingResource.equals(ResourceProfile.CPUS) && + defaultProf.maxTasksPerExecutor(sc.conf) < cpuSlots) { + throw new IllegalArgumentException("The number of slots on an executor has to be " + + "limited by the number of cores, otherwise you waste resources and " + + "some scheduling doesn't work properly. Your configuration has " + + s"core/task cpu slots = ${cpuSlots} and " + + s"${limitingResource} = " + + s"${defaultProf.maxTasksPerExecutor(sc.conf)}. Please adjust your configuration " + + "so that all resources require same number of executor slots.") } + ResourceUtils.warnOnWastedResources(defaultProf, sc.conf, Some(executorCores)) } master match { case "local" => - checkResourcesPerTask(clusterMode = false, Some(1)) + checkResourcesPerTask(1) val scheduler = new TaskSchedulerImpl(sc, MAX_LOCAL_TASK_FAILURES, isLocal = true) val backend = new LocalSchedulerBackend(sc.getConf, scheduler, 1) scheduler.initialize(backend) @@ -2886,7 +2822,7 @@ object SparkContext extends Logging { if (threadCount <= 0) { throw new SparkException(s"Asked to run locally with $threadCount threads") } - checkResourcesPerTask(clusterMode = false, Some(threadCount)) + checkResourcesPerTask(threadCount) val scheduler = new TaskSchedulerImpl(sc, MAX_LOCAL_TASK_FAILURES, isLocal = true) val backend = new LocalSchedulerBackend(sc.getConf, scheduler, threadCount) scheduler.initialize(backend) @@ -2897,14 +2833,13 @@ object SparkContext extends Logging { // local[*, M] means the number of cores on the computer with M failures // local[N, M] means exactly N threads with M failures val threadCount = if (threads == "*") localCpuCount else threads.toInt - checkResourcesPerTask(clusterMode = false, Some(threadCount)) + checkResourcesPerTask(threadCount) val scheduler = new TaskSchedulerImpl(sc, maxFailures.toInt, isLocal = true) val backend = new LocalSchedulerBackend(sc.getConf, scheduler, threadCount) scheduler.initialize(backend) (backend, scheduler) case SPARK_REGEX(sparkUrl) => - checkResourcesPerTask(clusterMode = true, None) val scheduler = new TaskSchedulerImpl(sc) val masterUrls = sparkUrl.split(",").map("spark://" + _) val backend = new StandaloneSchedulerBackend(scheduler, sc, masterUrls) @@ -2912,7 +2847,7 @@ object SparkContext extends Logging { (backend, scheduler) case LOCAL_CLUSTER_REGEX(numSlaves, coresPerSlave, memoryPerSlave) => - checkResourcesPerTask(clusterMode = true, Some(coresPerSlave.toInt)) + checkResourcesPerTask(coresPerSlave.toInt) // Check to make sure memory requested <= memoryPerSlave. Otherwise Spark will just hang. val memoryPerSlaveInt = memoryPerSlave.toInt if (sc.executorMemory > memoryPerSlaveInt) { @@ -2941,7 +2876,6 @@ object SparkContext extends Logging { (backend, scheduler) case masterUrl => - checkResourcesPerTask(clusterMode = true, None) val cm = getClusterManager(masterUrl) match { case Some(clusterMgr) => clusterMgr case None => throw new SparkException("Could not parse Master URL: '" + master + "'") diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala index 658e0d593a167..fa8bf0fc06358 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala @@ -24,8 +24,13 @@ import java.nio.charset.StandardCharsets.UTF_8 import java.util.concurrent.atomic.AtomicBoolean import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer import scala.util.control.NonFatal +import org.json4s.JsonAST._ +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods.{compact, render} + import org.apache.spark._ import org.apache.spark.internal.Logging import org.apache.spark.internal.config.{BUFFER_SIZE, EXECUTOR_CORES} @@ -238,13 +243,18 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( sock.setSoTimeout(10000) authHelper.authClient(sock) val input = new DataInputStream(sock.getInputStream()) - input.readInt() match { + val requestMethod = input.readInt() + // The BarrierTaskContext function may wait infinitely, socket shall not timeout + // before the function finishes. + sock.setSoTimeout(0) + requestMethod match { case BarrierTaskContextMessageProtocol.BARRIER_FUNCTION => - // The barrier() function may wait infinitely, socket shall not timeout - // before the function finishes. - sock.setSoTimeout(0) - barrierAndServe(sock) - + barrierAndServe(requestMethod, sock) + case BarrierTaskContextMessageProtocol.ALL_GATHER_FUNCTION => + val length = input.readInt() + val message = new Array[Byte](length) + input.readFully(message) + barrierAndServe(requestMethod, sock, new String(message, UTF_8)) case _ => val out = new DataOutputStream(new BufferedOutputStream( sock.getOutputStream)) @@ -395,15 +405,31 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( } /** - * Gateway to call BarrierTaskContext.barrier(). + * Gateway to call BarrierTaskContext methods. */ - def barrierAndServe(sock: Socket): Unit = { - require(serverSocket.isDefined, "No available ServerSocket to redirect the barrier() call.") - + def barrierAndServe(requestMethod: Int, sock: Socket, message: String = ""): Unit = { + require( + serverSocket.isDefined, + "No available ServerSocket to redirect the BarrierTaskContext method call." + ) val out = new DataOutputStream(new BufferedOutputStream(sock.getOutputStream)) try { - context.asInstanceOf[BarrierTaskContext].barrier() - writeUTF(BarrierTaskContextMessageProtocol.BARRIER_RESULT_SUCCESS, out) + var result: String = "" + requestMethod match { + case BarrierTaskContextMessageProtocol.BARRIER_FUNCTION => + context.asInstanceOf[BarrierTaskContext].barrier() + result = BarrierTaskContextMessageProtocol.BARRIER_RESULT_SUCCESS + case BarrierTaskContextMessageProtocol.ALL_GATHER_FUNCTION => + val messages: ArrayBuffer[String] = context.asInstanceOf[BarrierTaskContext].allGather( + message + ) + result = compact(render(JArray( + messages.map( + (message) => JString(message) + ).toList + ))) + } + writeUTF(result, out) } catch { case e: SparkException => writeUTF(e.getMessage, out) @@ -638,6 +664,7 @@ private[spark] object SpecialLengths { private[spark] object BarrierTaskContextMessageProtocol { val BARRIER_FUNCTION = 1 + val ALL_GATHER_FUNCTION = 2 val BARRIER_RESULT_SUCCESS = "success" val ERROR_UNRECOGNIZED_FUNCTION = "Not recognized function call from python side." } diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala index 62d60475985b3..490b48719b6be 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala @@ -27,7 +27,7 @@ import org.apache.spark.SparkContext import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} private[spark] object PythonUtils { - val PY4J_ZIP_NAME = "py4j-0.10.8.1-src.zip" + val PY4J_ZIP_NAME = "py4j-0.10.9-src.zip" /** Get the PYTHONPATH for PySpark, either from SPARK_HOME, if it is set, or from our JAR */ def sparkPythonPath: String = { diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala index fba371dcfb761..18305ad3746a6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala @@ -60,6 +60,15 @@ private[deploy] object DeployMessages { assert (port > 0) } + /** + * @param id the worker id + * @param worker the worker endpoint ref + */ + case class WorkerDecommission( + id: String, + worker: RpcEndpointRef) + extends DeployMessage + case class ExecutorStateChanged( appId: String, execId: Int, @@ -149,6 +158,8 @@ private[deploy] object DeployMessages { case object ReregisterWithMaster // used when a worker attempts to reconnect to a master + case object DecommissionSelf // Mark as decommissioned. May be Master to Worker in the future. + // AppClient to Master case class RegisterApplication(appDescription: ApplicationDescription, driver: RpcEndpointRef) diff --git a/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala b/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala index 69c98e28931d7..0751bcf221f86 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala @@ -19,9 +19,13 @@ package org.apache.spark.deploy private[deploy] object ExecutorState extends Enumeration { - val LAUNCHING, RUNNING, KILLED, FAILED, LOST, EXITED = Value + val LAUNCHING, RUNNING, KILLED, FAILED, LOST, EXITED, DECOMMISSIONED = Value type ExecutorState = Value - def isFinished(state: ExecutorState): Boolean = Seq(KILLED, FAILED, LOST, EXITED).contains(state) + // DECOMMISSIONED isn't listed as finished since we don't want to remove the executor from + // the worker and the executor still exists - but we do want to avoid scheduling new tasks on it. + private val finishedStates = Seq(KILLED, FAILED, LOST, EXITED) + + def isFinished(state: ExecutorState): Boolean = finishedStates.contains(state) } diff --git a/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala index 8f17159228f8b..eedf5e969e291 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClient.scala @@ -180,6 +180,8 @@ private[spark] class StandaloneAppClient( logInfo("Executor updated: %s is now %s%s".format(fullId, state, messageText)) if (ExecutorState.isFinished(state)) { listener.executorRemoved(fullId, message.getOrElse(""), exitStatus, workerLost) + } else if (state == ExecutorState.DECOMMISSIONED) { + listener.executorDecommissioned(fullId, message.getOrElse("")) } case WorkerRemoved(id, host, message) => diff --git a/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClientListener.scala b/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClientListener.scala index d8bc1a883def1..2e38a6847891d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClientListener.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/StandaloneAppClientListener.scala @@ -39,5 +39,7 @@ private[spark] trait StandaloneAppClientListener { def executorRemoved( fullId: String, message: String, exitStatus: Option[Int], workerLost: Boolean): Unit + def executorDecommissioned(fullId: String, message: String): Unit + def workerRemoved(workerId: String, host: String, message: String): Unit } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 8d3795cae707a..71df5dfa423a9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -243,6 +243,15 @@ private[deploy] class Master( logError("Leadership has been revoked -- master shutting down.") System.exit(0) + case WorkerDecommission(id, workerRef) => + logInfo("Recording worker %s decommissioning".format(id)) + if (state == RecoveryState.STANDBY) { + workerRef.send(MasterInStandby) + } else { + // We use foreach since get gives us an option and we can skip the failures. + idToWorker.get(id).foreach(decommissionWorker) + } + case RegisterWorker( id, workerHost, workerPort, workerRef, cores, memory, workerWebUiUrl, masterAddress, resources) => @@ -313,7 +322,9 @@ private[deploy] class Master( // Only retry certain number of times so we don't go into an infinite loop. // Important note: this code path is not exercised by tests, so be very careful when // changing this `if` condition. + // We also don't count failures from decommissioned workers since they are "expected." if (!normalExit + && oldState != ExecutorState.DECOMMISSIONED && appInfo.incrementRetryCount() >= maxExecutorRetries && maxExecutorRetries >= 0) { // < 0 disables this application-killing path val execs = appInfo.executors.values @@ -850,6 +861,26 @@ private[deploy] class Master( true } + private def decommissionWorker(worker: WorkerInfo): Unit = { + if (worker.state != WorkerState.DECOMMISSIONED) { + logInfo("Decommissioning worker %s on %s:%d".format(worker.id, worker.host, worker.port)) + worker.setState(WorkerState.DECOMMISSIONED) + for (exec <- worker.executors.values) { + logInfo("Telling app of decommission executors") + exec.application.driver.send(ExecutorUpdated( + exec.id, ExecutorState.DECOMMISSIONED, + Some("worker decommissioned"), None, workerLost = false)) + exec.state = ExecutorState.DECOMMISSIONED + exec.application.removeExecutor(exec) + } + // On recovery do not add a decommissioned executor + persistenceEngine.removeWorker(worker) + } else { + logWarning("Skipping decommissioning worker %s on %s:%d as worker is already decommissioned". + format(worker.id, worker.host, worker.port)) + } + } + private def removeWorker(worker: WorkerInfo, msg: String): Unit = { logInfo("Removing worker " + worker.id + " on " + worker.host + ":" + worker.port) worker.setState(WorkerState.DEAD) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 4be495ac4f13f..d988bcedb47f0 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -67,6 +67,14 @@ private[deploy] class Worker( Utils.checkHost(host) assert (port > 0) + // If worker decommissioning is enabled register a handler on PWR to shutdown. + if (conf.get(WORKER_DECOMMISSION_ENABLED)) { + logInfo("Registering SIGPWR handler to trigger decommissioning.") + SignalUtils.register("PWR")(decommissionSelf) + } else { + logInfo("Worker decommissioning not enabled, SIGPWR will result in exiting.") + } + // A scheduled executor used to send messages at the specified time. private val forwardMessageScheduler = ThreadUtils.newDaemonSingleThreadScheduledExecutor("worker-forward-message-scheduler") @@ -128,6 +136,7 @@ private[deploy] class Worker( private val workerUri = RpcEndpointAddress(rpcEnv.address, endpointName).toString private var registered = false private var connected = false + private var decommissioned = false private val workerId = generateWorkerId() private val sparkHome = if (sys.props.contains(IS_TESTING.key)) { @@ -549,6 +558,8 @@ private[deploy] class Worker( case LaunchExecutor(masterUrl, appId, execId, appDesc, cores_, memory_, resources_) => if (masterUrl != activeMasterUrl) { logWarning("Invalid Master (" + masterUrl + ") attempted to launch executor.") + } else if (decommissioned) { + logWarning("Asked to launch an executor while decommissioned. Not launching executor.") } else { try { logInfo("Asked to launch executor %s/%d for %s".format(appId, execId, appDesc.name)) @@ -672,6 +683,9 @@ private[deploy] class Worker( case ApplicationFinished(id) => finishedApps += id maybeCleanupApplication(id) + + case DecommissionSelf => + decommissionSelf() } override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { @@ -771,6 +785,18 @@ private[deploy] class Worker( } } + private[deploy] def decommissionSelf(): Boolean = { + if (conf.get(WORKER_DECOMMISSION_ENABLED)) { + logDebug("Decommissioning self") + decommissioned = true + sendToMaster(WorkerDecommission(workerId, self)) + } else { + logWarning("Asked to decommission self, but decommissioning not enabled") + } + // Return true since can be called as a signal handler + true + } + private[worker] def handleDriverStateChanged(driverStateChanged: DriverStateChanged): Unit = { val driverId = driverStateChanged.driverId val exception = driverStateChanged.exception diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 25c5b9812fa1a..faf03a64ae8b2 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -43,7 +43,7 @@ import org.apache.spark.rpc._ import org.apache.spark.scheduler.{ExecutorLossReason, TaskDescription} import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.serializer.SerializerInstance -import org.apache.spark.util.{ChildFirstURLClassLoader, MutableURLClassLoader, ThreadUtils, Utils} +import org.apache.spark.util.{ChildFirstURLClassLoader, MutableURLClassLoader, SignalUtils, ThreadUtils, Utils} private[spark] class CoarseGrainedExecutorBackend( override val rpcEnv: RpcEnv, @@ -64,6 +64,7 @@ private[spark] class CoarseGrainedExecutorBackend( private[this] val stopping = new AtomicBoolean(false) var executor: Executor = null + @volatile private var decommissioned = false @volatile var driver: Option[RpcEndpointRef] = None // If this CoarseGrainedExecutorBackend is changed to support multiple threads, then this may need @@ -80,6 +81,9 @@ private[spark] class CoarseGrainedExecutorBackend( private[executor] val taskResources = new mutable.HashMap[Long, Map[String, ResourceInformation]] override def onStart(): Unit = { + logInfo("Registering PWR handler.") + SignalUtils.register("PWR")(decommissionSelf) + logInfo("Connecting to driver: " + driverUrl) try { _resources = parseOrFindResources(resourcesFileOpt) @@ -160,6 +164,16 @@ private[spark] class CoarseGrainedExecutorBackend( if (executor == null) { exitExecutor(1, "Received LaunchTask command but executor was null") } else { + if (decommissioned) { + logError("Asked to launch a task while decommissioned.") + driver match { + case Some(endpoint) => + logInfo("Sending DecommissionExecutor to driver.") + endpoint.send(DecommissionExecutor(executorId)) + case _ => + logError("No registered driver to send Decommission to.") + } + } val taskDesc = TaskDescription.decode(data.value) logInfo("Got assigned task " + taskDesc.taskId) taskResources(taskDesc.taskId) = taskDesc.resources @@ -242,6 +256,29 @@ private[spark] class CoarseGrainedExecutorBackend( System.exit(code) } + + private def decommissionSelf(): Boolean = { + logInfo("Decommissioning self w/sync") + try { + decommissioned = true + // Tell master we are are decommissioned so it stops trying to schedule us + if (driver.nonEmpty) { + driver.get.askSync[Boolean](DecommissionExecutor(executorId)) + } else { + logError("No driver to message decommissioning.") + } + if (executor != null) { + executor.decommission() + } + logInfo("Done decommissioning self.") + // Return true since we are handling a signal + true + } catch { + case e: Exception => + logError(s"Error ${e} during attempt to decommission self") + false + } + } } private[spark] object CoarseGrainedExecutorBackend extends Logging { diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 8aeb16fe5d8c8..2bfa1cea4b26f 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -216,16 +216,32 @@ private[spark] class Executor( */ private var heartbeatFailures = 0 + /** + * Flag to prevent launching new tasks while decommissioned. There could be a race condition + * accessing this, but decommissioning is only intended to help not be a hard stop. + */ + private var decommissioned = false + heartbeater.start() metricsPoller.start() private[executor] def numRunningTasks: Int = runningTasks.size() + /** + * Mark an executor for decommissioning and avoid launching new tasks. + */ + private[spark] def decommission(): Unit = { + decommissioned = true + } + def launchTask(context: ExecutorBackend, taskDescription: TaskDescription): Unit = { val tr = new TaskRunner(context, taskDescription) runningTasks.put(taskDescription.taskId, tr) threadPool.execute(tr) + if (decommissioned) { + log.error(s"Launching a task while in decommissioned state.") + } } def killTask(taskId: Long, interruptThread: Boolean, reason: String): Unit = { diff --git a/core/src/main/scala/org/apache/spark/internal/Logging.scala b/core/src/main/scala/org/apache/spark/internal/Logging.scala index 2e4846bec2db4..0c1d9635b6535 100644 --- a/core/src/main/scala/org/apache/spark/internal/Logging.scala +++ b/core/src/main/scala/org/apache/spark/internal/Logging.scala @@ -117,7 +117,7 @@ trait Logging { } // For testing - def initializeForcefully(isInterpreter: Boolean, silent: Boolean): Unit = { + private[spark] def initializeForcefully(isInterpreter: Boolean, silent: Boolean): Unit = { initializeLogging(isInterpreter, silent) } diff --git a/core/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala b/core/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala index 68e1994f0f94f..8d5959a0c8b7f 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala @@ -129,7 +129,7 @@ private[spark] class TypedConfigBuilder[T]( def createOptional: OptionalConfigEntry[T] = { val entry = new OptionalConfigEntry[T](parent.key, parent._prependedKey, parent._prependSeparator, parent._alternatives, converter, stringConverter, parent._doc, - parent._public) + parent._public, parent._version) parent._onCreate.foreach(_(entry)) entry } @@ -144,7 +144,7 @@ private[spark] class TypedConfigBuilder[T]( val transformedDefault = converter(stringConverter(default)) val entry = new ConfigEntryWithDefault[T](parent.key, parent._prependedKey, parent._prependSeparator, parent._alternatives, transformedDefault, converter, - stringConverter, parent._doc, parent._public) + stringConverter, parent._doc, parent._public, parent._version) parent._onCreate.foreach(_(entry)) entry } @@ -154,7 +154,7 @@ private[spark] class TypedConfigBuilder[T]( def createWithDefaultFunction(defaultFunc: () => T): ConfigEntry[T] = { val entry = new ConfigEntryWithDefaultFunction[T](parent.key, parent._prependedKey, parent._prependSeparator, parent._alternatives, defaultFunc, converter, stringConverter, - parent._doc, parent._public) + parent._doc, parent._public, parent._version) parent._onCreate.foreach(_ (entry)) entry } @@ -166,7 +166,7 @@ private[spark] class TypedConfigBuilder[T]( def createWithDefaultString(default: String): ConfigEntry[T] = { val entry = new ConfigEntryWithDefaultString[T](parent.key, parent._prependedKey, parent._prependSeparator, parent._alternatives, default, converter, stringConverter, - parent._doc, parent._public) + parent._doc, parent._public, parent._version) parent._onCreate.foreach(_(entry)) entry } @@ -186,6 +186,7 @@ private[spark] case class ConfigBuilder(key: String) { private[config] var _prependSeparator: String = "" private[config] var _public = true private[config] var _doc = "" + private[config] var _version = "" private[config] var _onCreate: Option[ConfigEntry[_] => Unit] = None private[config] var _alternatives = List.empty[String] @@ -199,6 +200,11 @@ private[spark] case class ConfigBuilder(key: String) { this } + def version(v: String): ConfigBuilder = { + _version = v + this + } + /** * Registers a callback for when the config entry is finally instantiated. Currently used by * SQLConf to keep track of SQL configuration entries. @@ -255,7 +261,7 @@ private[spark] case class ConfigBuilder(key: String) { def fallbackConf[T](fallback: ConfigEntry[T]): ConfigEntry[T] = { val entry = new FallbackConfigEntry(key, _prependedKey, _prependSeparator, _alternatives, _doc, - _public, fallback) + _public, _version, fallback) _onCreate.foreach(_(entry)) entry } diff --git a/core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala b/core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala index c5df4c8820098..b98c7436f9906 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala @@ -17,6 +17,35 @@ package org.apache.spark.internal.config +// ==================================================================================== +// The guideline for naming configurations +// ==================================================================================== +/* +In general, the config name should be a noun that describes its basic purpose. It's +recommended to add prefix to the config name to make the scope clearer. For example, +`spark.scheduler.mode` clearly indicates that this config is for the scheduler. + +A config name can have multiple prefixes that are structured, which is similar to a +qualified Java class name. Each prefix behaves like a namespace. We should only create +a namespace if it's meaningful and can be shared by multiple configs. For example, +`buffer.inMemoryThreshold` is preferred over `buffer.in.memory.threshold`. + +The followings are best practices of naming configs for some common cases: +1. When adding configs for a big feature, it's better to create an umbrella config that + can turn the feature on/off, with a name like `featureName.enabled`. The other configs + of this feature should be put under the `featureName` namespace. For example: + - spark.sql.cbo.enabled + - spark.sql.cbo.starSchemaDetection + - spark.sql.cbo.joinReorder.enabled + - spark.sql.cbo.joinReorder.dp.threshold +2. When adding a boolean config, the name should be a verb that describes what + happens if this config is set to true, e.g. `spark.shuffle.sort.useRadixSort`. +3. When adding a config to specify a time duration, it's better to put the time unit + in the config name. For example, `featureName.timeoutMs`, which clearly indicates + that the time unit is millisecond. The config entry should be created with + `ConfigBuilder#timeConf`, to support time strings like `2 minutes`. +*/ + /** * An entry contains all meta information for a configuration. * @@ -39,6 +68,7 @@ package org.apache.spark.internal.config * @param doc the documentation for the configuration * @param isPublic if this configuration is public to the user. If it's `false`, this * configuration is only used internally and we should not expose it to users. + * @param version the spark version when the configuration was released. * @tparam T the value type */ private[spark] abstract class ConfigEntry[T] ( @@ -49,7 +79,8 @@ private[spark] abstract class ConfigEntry[T] ( val valueConverter: String => T, val stringConverter: T => String, val doc: String, - val isPublic: Boolean) { + val isPublic: Boolean, + val version: String) { import ConfigEntry._ @@ -74,7 +105,8 @@ private[spark] abstract class ConfigEntry[T] ( def defaultValue: Option[T] = None override def toString: String = { - s"ConfigEntry(key=$key, defaultValue=$defaultValueString, doc=$doc, public=$isPublic)" + s"ConfigEntry(key=$key, defaultValue=$defaultValueString, doc=$doc, " + + s"public=$isPublic, version=$version)" } } @@ -87,7 +119,8 @@ private class ConfigEntryWithDefault[T] ( valueConverter: String => T, stringConverter: T => String, doc: String, - isPublic: Boolean) + isPublic: Boolean, + version: String) extends ConfigEntry( key, prependedKey, @@ -96,7 +129,8 @@ private class ConfigEntryWithDefault[T] ( valueConverter, stringConverter, doc, - isPublic + isPublic, + version ) { override def defaultValue: Option[T] = Some(_defaultValue) @@ -117,7 +151,8 @@ private class ConfigEntryWithDefaultFunction[T] ( valueConverter: String => T, stringConverter: T => String, doc: String, - isPublic: Boolean) + isPublic: Boolean, + version: String) extends ConfigEntry( key, prependedKey, @@ -126,7 +161,8 @@ private class ConfigEntryWithDefaultFunction[T] ( valueConverter, stringConverter, doc, - isPublic + isPublic, + version ) { override def defaultValue: Option[T] = Some(_defaultFunction()) @@ -147,7 +183,8 @@ private class ConfigEntryWithDefaultString[T] ( valueConverter: String => T, stringConverter: T => String, doc: String, - isPublic: Boolean) + isPublic: Boolean, + version: String) extends ConfigEntry( key, prependedKey, @@ -156,7 +193,8 @@ private class ConfigEntryWithDefaultString[T] ( valueConverter, stringConverter, doc, - isPublic + isPublic, + version ) { override def defaultValue: Option[T] = Some(valueConverter(_defaultValue)) @@ -181,7 +219,8 @@ private[spark] class OptionalConfigEntry[T]( val rawValueConverter: String => T, val rawStringConverter: T => String, doc: String, - isPublic: Boolean) + isPublic: Boolean, + version: String) extends ConfigEntry[Option[T]]( key, prependedKey, @@ -190,7 +229,8 @@ private[spark] class OptionalConfigEntry[T]( s => Some(rawValueConverter(s)), v => v.map(rawStringConverter).orNull, doc, - isPublic + isPublic, + version ) { override def defaultValueString: String = ConfigEntry.UNDEFINED @@ -210,6 +250,7 @@ private[spark] class FallbackConfigEntry[T] ( alternatives: List[String], doc: String, isPublic: Boolean, + version: String, val fallback: ConfigEntry[T]) extends ConfigEntry[T]( key, @@ -219,7 +260,8 @@ private[spark] class FallbackConfigEntry[T] ( fallback.valueConverter, fallback.stringConverter, doc, - isPublic + isPublic, + version ) { override def defaultValueString: String = s"" diff --git a/core/src/main/scala/org/apache/spark/internal/config/Deploy.scala b/core/src/main/scala/org/apache/spark/internal/config/Deploy.scala index ceab957b36634..d494c5ec019c7 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/Deploy.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/Deploy.scala @@ -19,48 +19,59 @@ package org.apache.spark.internal.config private[spark] object Deploy { val RECOVERY_MODE = ConfigBuilder("spark.deploy.recoveryMode") + .version("0.8.1") .stringConf .createWithDefault("NONE") val RECOVERY_MODE_FACTORY = ConfigBuilder("spark.deploy.recoveryMode.factory") + .version("1.2.0") .stringConf .createWithDefault("") val RECOVERY_DIRECTORY = ConfigBuilder("spark.deploy.recoveryDirectory") + .version("0.8.1") .stringConf .createWithDefault("") val ZOOKEEPER_URL = ConfigBuilder("spark.deploy.zookeeper.url") .doc(s"When `${RECOVERY_MODE.key}` is set to ZOOKEEPER, this " + "configuration is used to set the zookeeper URL to connect to.") + .version("0.8.1") .stringConf .createOptional val ZOOKEEPER_DIRECTORY = ConfigBuilder("spark.deploy.zookeeper.dir") + .version("0.8.1") .stringConf .createOptional val RETAINED_APPLICATIONS = ConfigBuilder("spark.deploy.retainedApplications") + .version("0.8.0") .intConf .createWithDefault(200) val RETAINED_DRIVERS = ConfigBuilder("spark.deploy.retainedDrivers") + .version("1.1.0") .intConf .createWithDefault(200) val REAPER_ITERATIONS = ConfigBuilder("spark.dead.worker.persistence") + .version("0.8.0") .intConf .createWithDefault(15) val MAX_EXECUTOR_RETRIES = ConfigBuilder("spark.deploy.maxExecutorRetries") + .version("1.6.3") .intConf .createWithDefault(10) val SPREAD_OUT_APPS = ConfigBuilder("spark.deploy.spreadOut") + .version("0.6.1") .booleanConf .createWithDefault(true) val DEFAULT_CORES = ConfigBuilder("spark.deploy.defaultCores") + .version("0.9.0") .intConf .createWithDefault(Int.MaxValue) diff --git a/core/src/main/scala/org/apache/spark/internal/config/History.scala b/core/src/main/scala/org/apache/spark/internal/config/History.scala index 14fb5ff075472..8f99908507ceb 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/History.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/History.scala @@ -162,7 +162,7 @@ private[spark] object History { val APPLY_CUSTOM_EXECUTOR_LOG_URL_TO_INCOMPLETE_APP = ConfigBuilder("spark.history.custom.executor.log.url.applyIncompleteApplication") .doc("Whether to apply custom executor log url, as specified by " + - "`spark.history.custom.executor.log.url`, to incomplete application as well. " + + s"${CUSTOM_EXECUTOR_LOG_URL.key}, to incomplete application as well. " + "Even if this is true, this still only affects the behavior of the history server, " + "not running spark applications.") .booleanConf diff --git a/core/src/main/scala/org/apache/spark/internal/config/Network.scala b/core/src/main/scala/org/apache/spark/internal/config/Network.scala index 129e31a82979f..0961d062cc04f 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/Network.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/Network.scala @@ -23,71 +23,85 @@ private[spark] object Network { private[spark] val NETWORK_CRYPTO_SASL_FALLBACK = ConfigBuilder("spark.network.crypto.saslFallback") + .version("2.2.0") .booleanConf .createWithDefault(true) private[spark] val NETWORK_CRYPTO_ENABLED = ConfigBuilder("spark.network.crypto.enabled") + .version("2.2.0") .booleanConf .createWithDefault(false) private[spark] val NETWORK_REMOTE_READ_NIO_BUFFER_CONVERSION = ConfigBuilder("spark.network.remoteReadNioBufferConversion") + .version("2.4.0") .booleanConf .createWithDefault(false) private[spark] val NETWORK_TIMEOUT = ConfigBuilder("spark.network.timeout") + .version("1.3.0") .timeConf(TimeUnit.SECONDS) .createWithDefaultString("120s") private[spark] val NETWORK_TIMEOUT_INTERVAL = ConfigBuilder("spark.network.timeoutInterval") + .version("1.3.2") .timeConf(TimeUnit.MILLISECONDS) .createWithDefaultString(STORAGE_BLOCKMANAGER_TIMEOUTINTERVAL.defaultValueString) private[spark] val RPC_ASK_TIMEOUT = ConfigBuilder("spark.rpc.askTimeout") + .version("1.4.0") .stringConf .createOptional private[spark] val RPC_CONNECT_THREADS = ConfigBuilder("spark.rpc.connect.threads") + .version("1.6.0") .intConf .createWithDefault(64) private[spark] val RPC_IO_NUM_CONNECTIONS_PER_PEER = ConfigBuilder("spark.rpc.io.numConnectionsPerPeer") + .version("1.6.0") .intConf .createWithDefault(1) private[spark] val RPC_IO_THREADS = ConfigBuilder("spark.rpc.io.threads") + .version("1.6.0") .intConf .createOptional private[spark] val RPC_LOOKUP_TIMEOUT = ConfigBuilder("spark.rpc.lookupTimeout") + .version("1.4.0") .stringConf .createOptional private[spark] val RPC_MESSAGE_MAX_SIZE = ConfigBuilder("spark.rpc.message.maxSize") + .version("2.0.0") .intConf .createWithDefault(128) private[spark] val RPC_NETTY_DISPATCHER_NUM_THREADS = ConfigBuilder("spark.rpc.netty.dispatcher.numThreads") + .version("1.6.0") .intConf .createOptional private[spark] val RPC_NUM_RETRIES = ConfigBuilder("spark.rpc.numRetries") + .version("1.4.0") .intConf .createWithDefault(3) private[spark] val RPC_RETRY_WAIT = ConfigBuilder("spark.rpc.retry.wait") + .version("1.4.0") .timeConf(TimeUnit.MILLISECONDS) .createWithDefaultString("3s") } diff --git a/core/src/main/scala/org/apache/spark/internal/config/Python.scala b/core/src/main/scala/org/apache/spark/internal/config/Python.scala index 26a0598f49411..188d884319644 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/Python.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/Python.scala @@ -22,26 +22,32 @@ import org.apache.spark.network.util.ByteUnit private[spark] object Python { val PYTHON_WORKER_REUSE = ConfigBuilder("spark.python.worker.reuse") + .version("1.2.0") .booleanConf .createWithDefault(true) val PYTHON_TASK_KILL_TIMEOUT = ConfigBuilder("spark.python.task.killTimeout") + .version("2.2.2") .timeConf(TimeUnit.MILLISECONDS) .createWithDefaultString("2s") val PYTHON_USE_DAEMON = ConfigBuilder("spark.python.use.daemon") + .version("2.3.0") .booleanConf .createWithDefault(true) val PYTHON_DAEMON_MODULE = ConfigBuilder("spark.python.daemon.module") + .version("2.4.0") .stringConf .createOptional val PYTHON_WORKER_MODULE = ConfigBuilder("spark.python.worker.module") + .version("2.4.0") .stringConf .createOptional val PYSPARK_EXECUTOR_MEMORY = ConfigBuilder("spark.executor.pyspark.memory") + .version("2.4.0") .bytesConf(ByteUnit.MiB) .createOptional } diff --git a/core/src/main/scala/org/apache/spark/internal/config/R.scala b/core/src/main/scala/org/apache/spark/internal/config/R.scala index 26e06a5231c42..46fc198cd4cf5 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/R.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/R.scala @@ -19,22 +19,27 @@ package org.apache.spark.internal.config private[spark] object R { val R_BACKEND_CONNECTION_TIMEOUT = ConfigBuilder("spark.r.backendConnectionTimeout") + .version("2.1.0") .intConf .createWithDefault(6000) val R_NUM_BACKEND_THREADS = ConfigBuilder("spark.r.numRBackendThreads") + .version("1.4.0") .intConf .createWithDefault(2) val R_HEARTBEAT_INTERVAL = ConfigBuilder("spark.r.heartBeatInterval") + .version("2.1.0") .intConf .createWithDefault(100) val SPARKR_COMMAND = ConfigBuilder("spark.sparkr.r.command") + .version("1.5.3") .stringConf .createWithDefault("Rscript") val R_COMMAND = ConfigBuilder("spark.r.command") + .version("1.5.3") .stringConf .createOptional } diff --git a/core/src/main/scala/org/apache/spark/internal/config/Tests.scala b/core/src/main/scala/org/apache/spark/internal/config/Tests.scala index 21660ab3a9512..51df73ebde07d 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/Tests.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/Tests.scala @@ -53,4 +53,13 @@ private[spark] object Tests { val TEST_N_CORES_EXECUTOR = ConfigBuilder("spark.testing.nCoresPerExecutor") .intConf .createWithDefault(2) + + val RESOURCES_WARNING_TESTING = + ConfigBuilder("spark.resources.warnings.testing").booleanConf.createWithDefault(false) + + val RESOURCE_PROFILE_MANAGER_TESTING = + ConfigBuilder("spark.testing.resourceProfileManager") + .booleanConf + .createWithDefault(false) + } diff --git a/core/src/main/scala/org/apache/spark/internal/config/Worker.scala b/core/src/main/scala/org/apache/spark/internal/config/Worker.scala index f1eaae29f18df..2b175c1e14ee5 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/Worker.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/Worker.scala @@ -71,4 +71,9 @@ private[spark] object Worker { ConfigBuilder("spark.worker.ui.compressedLogFileLengthCacheSize") .intConf .createWithDefault(100) + + private[spark] val WORKER_DECOMMISSION_ENABLED = + ConfigBuilder("spark.worker.decommission.enabled") + .booleanConf + .createWithDefault(false) } diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 02acb6b530737..37ce178407381 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -38,7 +38,7 @@ package object config { private[spark] val LISTENER_BUS_EVENT_QUEUE_PREFIX = "spark.scheduler.listenerbus.eventqueue" private[spark] val SPARK_RESOURCES_COORDINATE = - ConfigBuilder("spark.resources.coordinate.enable") + ConfigBuilder("spark.resources.coordinateResourcesInStandalone") .doc("Whether to coordinate resources automatically among workers/drivers(client only) " + "in Standalone. If false, the user is responsible for configuring different resources " + "for workers/drivers that run on the same host.") @@ -55,7 +55,7 @@ package object config { .createOptional private[spark] val RESOURCES_DISCOVERY_PLUGIN = - ConfigBuilder("spark.resources.discovery.plugin") + ConfigBuilder("spark.resources.discoveryPlugin") .doc("Comma-separated list of class names implementing" + "org.apache.spark.api.resource.ResourceDiscoveryPlugin to load into the application." + "This is for advanced users to replace the resource discovery class with a " + @@ -159,7 +159,7 @@ package object config { .createWithDefaultString("100k") private[spark] val EVENT_LOG_STAGE_EXECUTOR_METRICS = - ConfigBuilder("spark.eventLog.logStageExecutorMetrics.enabled") + ConfigBuilder("spark.eventLog.logStageExecutorMetrics") .doc("Whether to write per-stage peaks of executor metrics (for each executor) " + "to the event log.") .booleanConf @@ -191,14 +191,15 @@ package object config { private[spark] val EVENT_LOG_ENABLE_ROLLING = ConfigBuilder("spark.eventLog.rolling.enabled") - .doc("Whether rolling over event log files is enabled. If set to true, it cuts down " + + .doc("Whether rolling over event log files is enabled. If set to true, it cuts down " + "each event log file to the configured size.") .booleanConf .createWithDefault(false) private[spark] val EVENT_LOG_ROLLING_MAX_FILE_SIZE = ConfigBuilder("spark.eventLog.rolling.maxFileSize") - .doc("The max size of event log file to be rolled over.") + .doc(s"When ${EVENT_LOG_ENABLE_ROLLING.key}=true, specifies the max size of event log file" + + " to be rolled over.") .bytesConf(ByteUnit.BYTE) .checkValue(_ >= ByteUnit.MiB.toBytes(10), "Max file size of event log should be " + "configured to be at least 10 MiB.") @@ -631,7 +632,7 @@ package object config { .createWithDefault(128) private[spark] val LISTENER_BUS_LOG_SLOW_EVENT_ENABLED = - ConfigBuilder("spark.scheduler.listenerbus.logSlowEvent.enabled") + ConfigBuilder("spark.scheduler.listenerbus.logSlowEvent") .internal() .doc("When enabled, log the event that takes too much time to process. This helps us " + "discover the event types that cause performance bottlenecks. The time threshold is " + @@ -643,7 +644,7 @@ package object config { ConfigBuilder("spark.scheduler.listenerbus.logSlowEvent.threshold") .internal() .doc("The time threshold of whether a event is considered to be taking too much time to " + - "process. Log the event if spark.scheduler.listenerbus.logSlowEvent.enabled is true.") + s"process. Log the event if ${LISTENER_BUS_LOG_SLOW_EVENT_ENABLED.key} is true.") .timeConf(TimeUnit.NANOSECONDS) .createWithDefaultString("1s") @@ -1114,16 +1115,6 @@ package object config { .booleanConf .createWithDefault(false) - private[spark] val STORAGE_LOCAL_DISK_BY_EXECUTORS_CACHE_SIZE = - ConfigBuilder("spark.storage.localDiskByExecutors.cacheSize") - .doc("The max number of executors for which the local dirs are stored. This size is " + - "both applied for the driver and both for the executors side to avoid having an " + - "unbounded store. This cache will be used to avoid the network in case of fetching disk " + - "persisted RDD blocks or shuffle blocks (when `spark.shuffle.readHostLocalDisk.enabled` " + - "is set) from the same host.") - .intConf - .createWithDefault(1000) - private[spark] val SHUFFLE_SYNC = ConfigBuilder("spark.shuffle.sync") .doc("Whether to force outstanding writes to disk.") @@ -1160,13 +1151,23 @@ package object config { .createWithDefault(false) private[spark] val SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED = - ConfigBuilder("spark.shuffle.readHostLocalDisk.enabled") + ConfigBuilder("spark.shuffle.readHostLocalDisk") .doc(s"If enabled (and `${SHUFFLE_USE_OLD_FETCH_PROTOCOL.key}` is disabled), shuffle " + "blocks requested from those block managers which are running on the same host are read " + "from the disk directly instead of being fetched as remote blocks over the network.") .booleanConf .createWithDefault(true) + private[spark] val STORAGE_LOCAL_DISK_BY_EXECUTORS_CACHE_SIZE = + ConfigBuilder("spark.storage.localDiskByExecutors.cacheSize") + .doc("The max number of executors for which the local dirs are stored. This size is " + + "both applied for the driver and both for the executors side to avoid having an " + + "unbounded store. This cache will be used to avoid the network in case of fetching disk " + + s"persisted RDD blocks or shuffle blocks " + + s"(when `${SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED.key}` is set) from the same host.") + .intConf + .createWithDefault(1000) + private[spark] val MEMORY_MAP_LIMIT_FOR_TESTS = ConfigBuilder("spark.storage.memoryMapLimitForTests") .internal() diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala index b3904f3362e8e..62726f7e147c5 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala @@ -105,8 +105,14 @@ class NettyBlockRpcServer( val blockId = BlockId(uploadBlock.blockId) logDebug(s"Receiving replicated block $blockId with level ${level} " + s"from ${client.getSocketAddress}") - blockManager.putBlockData(blockId, data, level, classTag) - responseContext.onSuccess(ByteBuffer.allocate(0)) + val blockStored = blockManager.putBlockData(blockId, data, level, classTag) + if (blockStored) { + responseContext.onSuccess(ByteBuffer.allocate(0)) + } else { + val exception = new Exception(s"Upload block for $blockId failed. This mostly happens " + + s"when there is not sufficient space available to store the block.") + responseContext.onFailure(exception) + } } } diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 64d2032a12721..a26b5791fa08b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -361,6 +361,7 @@ abstract class RDD[T: ClassTag]( readCachedBlock = false computeOrReadCheckpoint(partition, context) }) match { + // Block hit. case Left(blockResult) => if (readCachedBlock) { val existingMetrics = context.taskMetrics().inputMetrics @@ -374,6 +375,7 @@ abstract class RDD[T: ClassTag]( } else { new InterruptibleIterator(context, blockResult.data.asInstanceOf[Iterator[T]]) } + // Need to compute the block. case Right(iter) => new InterruptibleIterator(context, iter.asInstanceOf[Iterator[T]]) } diff --git a/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala b/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala index d345674d6635c..d4c29f9a70c44 100644 --- a/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala +++ b/core/src/main/scala/org/apache/spark/resource/ExecutorResourceRequests.scala @@ -109,7 +109,7 @@ private[spark] class ExecutorResourceRequests() extends Serializable { discoveryScript: String = "", vendor: String = ""): this.type = { // a bit weird but for Java api use empty string as meaning None because empty - // string is otherwise invalid for those paramters anyway + // string is otherwise invalid for those parameters anyway val req = new ExecutorResourceRequest(resourceName, amount, discoveryScript, vendor) _executorResources.put(resourceName, req) this diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceDiscoveryScriptPlugin.scala b/core/src/main/scala/org/apache/spark/resource/ResourceDiscoveryScriptPlugin.scala index 2ac6d3c500f9d..7027d1e3511b5 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceDiscoveryScriptPlugin.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceDiscoveryScriptPlugin.scala @@ -21,6 +21,7 @@ import java.io.File import java.util.Optional import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.resource.ResourceDiscoveryPlugin import org.apache.spark.internal.Logging import org.apache.spark.util.Utils.executeAndGetOutput @@ -32,6 +33,7 @@ import org.apache.spark.util.Utils.executeAndGetOutput * If the user specifies custom plugins, this is the last one to be executed and * throws if the resource isn't discovered. */ +@DeveloperApi class ResourceDiscoveryScriptPlugin extends ResourceDiscoveryPlugin with Logging { override def discoverResource( request: ResourceRequest, diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala index 14019d27fc2e6..844026d246e2c 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfile.scala @@ -22,12 +22,14 @@ import java.util.concurrent.atomic.AtomicInteger import javax.annotation.concurrent.GuardedBy import scala.collection.JavaConverters._ +import scala.collection.mutable -import org.apache.spark.SparkConf +import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.annotation.Evolving import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Python.PYSPARK_EXECUTOR_MEMORY +import org.apache.spark.util.Utils /** * Resource profile to associate with an RDD. A ResourceProfile allows the user to @@ -42,6 +44,13 @@ class ResourceProfile( // _id is only a var for testing purposes private var _id = ResourceProfile.getNextProfileId + // This is used for any resources that use fractional amounts, the key is the resource name + // and the value is the number of tasks that can share a resource address. For example, + // if the user says task gpu amount is 0.5, that results in 2 tasks per resource address. + private var _executorResourceSlotsPerAddr: Option[Map[String, Int]] = None + private var _limitingResource: Option[String] = None + private var _maxTasksPerExecutor: Option[Int] = None + private var _coresLimitKnown: Boolean = false def id: Int = _id @@ -67,6 +76,139 @@ class ResourceProfile( taskResources.get(ResourceProfile.CPUS).map(_.amount.toInt) } + private[spark] def getNumSlotsPerAddress(resource: String, sparkConf: SparkConf): Int = { + _executorResourceSlotsPerAddr.getOrElse { + calculateTasksAndLimitingResource(sparkConf) + } + _executorResourceSlotsPerAddr.get.getOrElse(resource, + throw new SparkException(s"Resource $resource doesn't exist in profile id: $id")) + } + + // Maximum tasks you could put on an executor with this profile based on the limiting resource. + // If the executor cores config is not present this value is based on the other resources + // available or 1 if no other resources. You need to check the isCoresLimitKnown to + // calculate proper value. + private[spark] def maxTasksPerExecutor(sparkConf: SparkConf): Int = { + _maxTasksPerExecutor.getOrElse { + calculateTasksAndLimitingResource(sparkConf) + _maxTasksPerExecutor.get + } + } + + // Returns whether the executor cores was available to use to calculate the max tasks + // per executor and limiting resource. Some cluster managers (like standalone and coarse + // grained mesos) don't use the cores config by default so we can't use it to calculate slots. + private[spark] def isCoresLimitKnown: Boolean = _coresLimitKnown + + // The resource that has the least amount of slots per executor. Its possible multiple or all + // resources result in same number of slots and this could be any of those. + // If the executor cores config is not present this value is based on the other resources + // available or empty string if no other resources. You need to check the isCoresLimitKnown to + // calculate proper value. + private[spark] def limitingResource(sparkConf: SparkConf): String = { + _limitingResource.getOrElse { + calculateTasksAndLimitingResource(sparkConf) + _limitingResource.get + } + } + + // executor cores config is not set for some masters by default and the default value + // only applies to yarn/k8s + private def shouldCheckExecutorCores(sparkConf: SparkConf): Boolean = { + val master = sparkConf.getOption("spark.master") + sparkConf.contains(EXECUTOR_CORES) || + (master.isDefined && (master.get.equalsIgnoreCase("yarn") || master.get.startsWith("k8s"))) + } + + /** + * Utility function to calculate the number of tasks you can run on a single Executor based + * on the task and executor resource requests in the ResourceProfile. This will be based + * off the resource that is most restrictive. For instance, if the executor + * request is for 4 cpus and 2 gpus and your task request is for 1 cpu and 1 gpu each, the + * limiting resource is gpu and the number of tasks you can run on a single executor is 2. + * This function also sets the limiting resource, isCoresLimitKnown and number of slots per + * resource address. + */ + private def calculateTasksAndLimitingResource(sparkConf: SparkConf): Unit = synchronized { + val shouldCheckExecCores = shouldCheckExecutorCores(sparkConf) + var (taskLimit, limitingResource) = if (shouldCheckExecCores) { + val cpusPerTask = taskResources.get(ResourceProfile.CPUS) + .map(_.amount).getOrElse(sparkConf.get(CPUS_PER_TASK).toDouble).toInt + assert(cpusPerTask > 0, "CPUs per task configuration has to be > 0") + val coresPerExecutor = getExecutorCores.getOrElse(sparkConf.get(EXECUTOR_CORES)) + _coresLimitKnown = true + ResourceUtils.validateTaskCpusLargeEnough(coresPerExecutor, cpusPerTask) + val tasksBasedOnCores = coresPerExecutor / cpusPerTask + // Note that if the cores per executor aren't set properly this calculation could be off, + // we default it to just be 1 in order to allow checking of the rest of the custom + // resources. We set the limit based on the other resources available. + (tasksBasedOnCores, ResourceProfile.CPUS) + } else { + (-1, "") + } + val numPartsPerResourceMap = new mutable.HashMap[String, Int] + numPartsPerResourceMap(ResourceProfile.CORES) = 1 + val taskResourcesToCheck = new mutable.HashMap[String, TaskResourceRequest] + taskResourcesToCheck ++= ResourceProfile.getCustomTaskResources(this) + val execResourceToCheck = ResourceProfile.getCustomExecutorResources(this) + execResourceToCheck.foreach { case (rName, execReq) => + val taskReq = taskResources.get(rName).map(_.amount).getOrElse(0.0) + numPartsPerResourceMap(rName) = 1 + if (taskReq > 0.0) { + if (taskReq > execReq.amount) { + throw new SparkException(s"The executor resource: $rName, amount: ${execReq.amount} " + + s"needs to be >= the task resource request amount of $taskReq") + } + val (numPerTask, parts) = ResourceUtils.calculateAmountAndPartsForFraction(taskReq) + numPartsPerResourceMap(rName) = parts + val numTasks = ((execReq.amount * parts) / numPerTask).toInt + if (taskLimit == -1 || numTasks < taskLimit) { + if (shouldCheckExecCores) { + // TODO - until resource profiles full implemented we need to error if cores not + // limiting resource because the scheduler code uses that for slots + throw new IllegalArgumentException("The number of slots on an executor has to be " + + "limited by the number of cores, otherwise you waste resources and " + + "some scheduling doesn't work properly. Your configuration has " + + s"core/task cpu slots = ${taskLimit} and " + + s"${execReq.resourceName} = ${numTasks}. " + + "Please adjust your configuration so that all resources require same number " + + "of executor slots.") + } + limitingResource = rName + taskLimit = numTasks + } + taskResourcesToCheck -= rName + } else { + logWarning(s"The executor resource config for resource: $rName was specified but " + + "no corresponding task resource request was specified.") + } + } + if(!shouldCheckExecCores && execResourceToCheck.nonEmpty) { + // if we can't rely on the executor cores config throw a warning for user + logWarning("Please ensure that the number of slots available on your " + + "executors is limited by the number of cores to task cpus and not another " + + "custom resource.") + } + if (taskResourcesToCheck.nonEmpty) { + throw new SparkException("No executor resource configs were not specified for the " + + s"following task configs: ${taskResourcesToCheck.keys.mkString(",")}") + } + val limiting = + if (taskLimit == -1) "cpu" else s"$limitingResource at $taskLimit tasks per executor" + logInfo(s"Limiting resource is $limiting") + _executorResourceSlotsPerAddr = Some(numPartsPerResourceMap.toMap) + _maxTasksPerExecutor = if (taskLimit == -1) Some(1) else Some(taskLimit) + _limitingResource = Some(limitingResource) + if (shouldCheckExecCores) { + ResourceUtils.warnOnWastedResources(this, sparkConf) + } + } + + // to be used only by history server for reconstruction from events + private[spark] def setResourceProfileId(id: Int): Unit = { + _id = id + } + // testing only private[spark] def setToDefaultProfile(): Unit = { _id = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID @@ -123,7 +265,7 @@ object ResourceProfile extends Logging { val taskResources = getDefaultTaskResources(conf) val executorResources = getDefaultExecutorResources(conf) val defProf = new ResourceProfile(executorResources, taskResources) - defProf.setToDefaultProfile + defProf.setToDefaultProfile() defaultProfile = Some(defProf) logInfo("Default ResourceProfile created, executor resources: " + s"${defProf.executorResources}, task resources: " + @@ -157,13 +299,12 @@ object ResourceProfile extends Logging { // for testing only private[spark] def reInitDefaultProfile(conf: SparkConf): Unit = { - clearDefaultProfile + clearDefaultProfile() // force recreate it after clearing getOrCreateDefaultProfile(conf) } - // for testing only - private[spark] def clearDefaultProfile: Unit = { + private[spark] def clearDefaultProfile(): Unit = { DEFAULT_PROFILE_LOCK.synchronized { defaultProfile = None } diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfileBuilder.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfileBuilder.scala index 0d55c176eeb65..26f23f4bf0476 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceProfileBuilder.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfileBuilder.scala @@ -31,7 +31,7 @@ import org.apache.spark.annotation.Evolving * requirements between stages. */ @Evolving -class ResourceProfileBuilder() { +private[spark] class ResourceProfileBuilder() { private val _taskResources = new ConcurrentHashMap[String, TaskResourceRequest]() private val _executorResources = new ConcurrentHashMap[String, ExecutorResourceRequest]() diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala b/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala new file mode 100644 index 0000000000000..06db9468c451e --- /dev/null +++ b/core/src/main/scala/org/apache/spark/resource/ResourceProfileManager.scala @@ -0,0 +1,86 @@ +/* + * 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.spark.resource + +import java.util.concurrent.ConcurrentHashMap + +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.annotation.Evolving +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Tests._ +import org.apache.spark.util.Utils +import org.apache.spark.util.Utils.isTesting + +/** + * Manager of resource profiles. The manager allows one place to keep the actual ResourceProfiles + * and everywhere else we can use the ResourceProfile Id to save on space. + * Note we never remove a resource profile at this point. Its expected this number if small + * so this shouldn't be much overhead. + */ +@Evolving +private[spark] class ResourceProfileManager(sparkConf: SparkConf) extends Logging { + private val resourceProfileIdToResourceProfile = new ConcurrentHashMap[Int, ResourceProfile]() + + private val defaultProfile = ResourceProfile.getOrCreateDefaultProfile(sparkConf) + addResourceProfile(defaultProfile) + + def defaultResourceProfile: ResourceProfile = defaultProfile + + private val taskCpusDefaultProfile = defaultProfile.getTaskCpus.get + private val dynamicEnabled = Utils.isDynamicAllocationEnabled(sparkConf) + private val master = sparkConf.getOption("spark.master") + private val isNotYarn = master.isDefined && !master.get.equals("yarn") + private val errorForTesting = !isTesting || sparkConf.get(RESOURCE_PROFILE_MANAGER_TESTING) + + // If we use anything except the default profile, its only supported on YARN right now. + // Throw an exception if not supported. + private[spark] def isSupported(rp: ResourceProfile): Boolean = { + val isNotDefaultProfile = rp.id != ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID + val notYarnAndNotDefaultProfile = isNotDefaultProfile && isNotYarn + val YarnNotDynAllocAndNotDefaultProfile = isNotDefaultProfile && !isNotYarn && !dynamicEnabled + if (errorForTesting && (notYarnAndNotDefaultProfile || YarnNotDynAllocAndNotDefaultProfile)) { + throw new SparkException("ResourceProfiles are only supported on YARN with dynamic " + + "allocation enabled.") + } + true + } + + def addResourceProfile(rp: ResourceProfile): Unit = { + isSupported(rp) + // force the computation of maxTasks and limitingResource now so we don't have cost later + rp.limitingResource(sparkConf) + logInfo(s"Adding ResourceProfile id: ${rp.id}") + resourceProfileIdToResourceProfile.putIfAbsent(rp.id, rp) + } + + /* + * Gets the ResourceProfile associated with the id, if a profile doesn't exist + * it returns the default ResourceProfile created from the application level configs. + */ + def resourceProfileFromId(rpId: Int): ResourceProfile = { + val rp = resourceProfileIdToResourceProfile.get(rpId) + if (rp == null) { + throw new SparkException(s"ResourceProfileId $rpId not found!") + } + rp + } + + def taskCpusForProfileId(rpId: Int): Int = { + resourceProfileFromId(rpId).getTaskCpus.getOrElse(taskCpusDefaultProfile) + } +} diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala b/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala index 7dd7fc1b99353..22272557aa00f 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala @@ -29,7 +29,8 @@ import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.resource.ResourceDiscoveryPlugin import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.{RESOURCES_DISCOVERY_PLUGIN, SPARK_TASK_PREFIX} +import org.apache.spark.internal.config.{CPUS_PER_TASK, EXECUTOR_CORES, RESOURCES_DISCOVERY_PLUGIN, SPARK_TASK_PREFIX} +import org.apache.spark.internal.config.Tests.{RESOURCES_WARNING_TESTING} import org.apache.spark.util.Utils /** @@ -161,19 +162,23 @@ private[spark] object ResourceUtils extends Logging { } // Used to take a fraction amount from a task resource requirement and split into a real - // integer amount and the number of parts expected. For instance, if the amount is 0.5, - // the we get (1, 2) back out. - // Returns tuple of (amount, numParts) - def calculateAmountAndPartsForFraction(amount: Double): (Int, Int) = { - val parts = if (amount <= 0.5) { - Math.floor(1.0 / amount).toInt - } else if (amount % 1 != 0) { + // integer amount and the number of slots per address. For instance, if the amount is 0.5, + // the we get (1, 2) back out. This indicates that for each 1 address, it has 2 slots per + // address, which allows you to put 2 tasks on that address. Note if amount is greater + // than 1, then the number of slots per address has to be 1. This would indicate that a + // would have multiple addresses assigned per task. This can be used for calculating + // the number of tasks per executor -> (executorAmount * numParts) / (integer amount). + // Returns tuple of (integer amount, numParts) + def calculateAmountAndPartsForFraction(doubleAmount: Double): (Int, Int) = { + val parts = if (doubleAmount <= 0.5) { + Math.floor(1.0 / doubleAmount).toInt + } else if (doubleAmount % 1 != 0) { throw new SparkException( - s"The resource amount ${amount} must be either <= 0.5, or a whole number.") + s"The resource amount ${doubleAmount} must be either <= 0.5, or a whole number.") } else { 1 } - (Math.ceil(amount).toInt, parts) + (Math.ceil(doubleAmount).toInt, parts) } // Add any task resource requests from the spark conf to the TaskResourceRequests passed in @@ -357,8 +362,13 @@ private[spark] object ResourceUtils extends Logging { def logResourceInfo(componentName: String, resources: Map[String, ResourceInformation]) : Unit = { + val resourceInfo = if (resources.isEmpty) { + s"No custom resources configured for $componentName." + } else { + s"Custom resources for $componentName:\n${resources.mkString("\n")}" + } logInfo("==============================================================") - logInfo(s"Resources for $componentName:\n${resources.mkString("\n")}") + logInfo(resourceInfo) logInfo("==============================================================") } @@ -382,6 +392,90 @@ private[spark] object ResourceUtils extends Logging { s"${resourceRequest.id.resourceName}") } + def validateTaskCpusLargeEnough(execCores: Int, taskCpus: Int): Boolean = { + // Number of cores per executor must meet at least one task requirement. + if (execCores < taskCpus) { + throw new SparkException(s"The number of cores per executor (=$execCores) has to be >= " + + s"the number of cpus per task = $taskCpus.") + } + true + } + + // the option executor cores parameter is by the different local modes since it not configured + // via the config + def warnOnWastedResources( + rp: ResourceProfile, + sparkConf: SparkConf, + execCores: Option[Int] = None): Unit = { + // There have been checks on the ResourceProfile to make sure the executor resources were + // specified and are large enough if any task resources were specified. + // Now just do some sanity test and log warnings when it looks like the user will + // waste some resources. + val coresKnown = rp.isCoresLimitKnown + var limitingResource = rp.limitingResource(sparkConf) + var maxTaskPerExec = rp.maxTasksPerExecutor(sparkConf) + val taskCpus = rp.getTaskCpus.getOrElse(sparkConf.get(CPUS_PER_TASK)) + val cores = if (execCores.isDefined) { + execCores.get + } else if (coresKnown) { + rp.getExecutorCores.getOrElse(sparkConf.get(EXECUTOR_CORES)) + } else { + // can't calculate cores limit + return + } + // when executor cores config isn't set, we can't calculate the real limiting resource + // and number of tasks per executor ahead of time, so calculate it now. + if (!coresKnown) { + val numTasksPerExecCores = cores / taskCpus + val numTasksPerExecCustomResource = rp.maxTasksPerExecutor(sparkConf) + if (limitingResource.isEmpty || + (limitingResource.nonEmpty && numTasksPerExecCores < numTasksPerExecCustomResource)) { + limitingResource = ResourceProfile.CPUS + maxTaskPerExec = numTasksPerExecCores + } + } + val taskReq = ResourceProfile.getCustomTaskResources(rp) + val execReq = ResourceProfile.getCustomExecutorResources(rp) + + if (limitingResource.nonEmpty && !limitingResource.equals(ResourceProfile.CPUS)) { + if ((taskCpus * maxTaskPerExec) < cores) { + val resourceNumSlots = Math.floor(cores/taskCpus).toInt + val message = s"The configuration of cores (exec = ${cores} " + + s"task = ${taskCpus}, runnable tasks = ${resourceNumSlots}) will " + + s"result in wasted resources due to resource ${limitingResource} limiting the " + + s"number of runnable tasks per executor to: ${maxTaskPerExec}. Please adjust " + + "your configuration." + if (sparkConf.get(RESOURCES_WARNING_TESTING)) { + throw new SparkException(message) + } else { + logWarning(message) + } + } + } + + taskReq.foreach { case (rName, treq) => + val execAmount = execReq(rName).amount + val numParts = rp.getNumSlotsPerAddress(rName, sparkConf) + // handle fractional + val taskAmount = if (numParts > 1) 1 else treq.amount + if (maxTaskPerExec < (execAmount * numParts / taskAmount)) { + val taskReqStr = s"${taskAmount}/${numParts}" + val resourceNumSlots = Math.floor(execAmount * numParts / taskAmount).toInt + val message = s"The configuration of resource: ${treq.resourceName} " + + s"(exec = ${execAmount}, task = ${taskReqStr}, " + + s"runnable tasks = ${resourceNumSlots}) will " + + s"result in wasted resources due to resource ${limitingResource} limiting the " + + s"number of runnable tasks per executor to: ${maxTaskPerExec}. Please adjust " + + "your configuration." + if (sparkConf.get(RESOURCES_WARNING_TESTING)) { + throw new SparkException(message) + } else { + logWarning(message) + } + } + } + } + // known types of resources final val GPU: String = "gpu" final val FPGA: String = "fpga" diff --git a/core/src/main/scala/org/apache/spark/rpc/RpcEndpointRef.scala b/core/src/main/scala/org/apache/spark/rpc/RpcEndpointRef.scala index 49d58929a97a4..56f3d377f8e2a 100644 --- a/core/src/main/scala/org/apache/spark/rpc/RpcEndpointRef.scala +++ b/core/src/main/scala/org/apache/spark/rpc/RpcEndpointRef.scala @@ -108,7 +108,7 @@ private[spark] abstract class RpcEndpointRef(conf: SparkConf) /** * An exception thrown if the RPC is aborted. */ -class RpcAbortException(message: String) extends Exception(message) +private[spark] class RpcAbortException(message: String) extends Exception(message) /** * A wrapper for [[Future]] but add abort method. diff --git a/core/src/main/scala/org/apache/spark/scheduler/AsyncEventQueue.scala b/core/src/main/scala/org/apache/spark/scheduler/AsyncEventQueue.scala index 1bcddaceb3576..5164c30fce0a1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/AsyncEventQueue.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/AsyncEventQueue.scala @@ -64,11 +64,14 @@ private class AsyncEventQueue( // processed (instead of just dequeued). private val eventCount = new AtomicLong() - /** A counter for dropped events. It will be reset every time we log it. */ + /** A counter for dropped events. */ private val droppedEventsCounter = new AtomicLong(0L) + /** A counter to keep number of dropped events last time it was logged */ + @volatile private var lastDroppedEventsCounter: Long = 0L + /** When `droppedEventsCounter` was logged last time in milliseconds. */ - @volatile private var lastReportTimestamp = 0L + private val lastReportTimestamp = new AtomicLong(0L) private val logDroppedEvent = new AtomicBoolean(false) @@ -167,21 +170,19 @@ private class AsyncEventQueue( } logTrace(s"Dropping event $event") - val droppedCount = droppedEventsCounter.get - if (droppedCount > 0) { - // Don't log too frequently - if (System.currentTimeMillis() - lastReportTimestamp >= 60 * 1000) { - // There may be multiple threads trying to decrease droppedEventsCounter. - // Use "compareAndSet" to make sure only one thread can win. - // And if another thread is increasing droppedEventsCounter, "compareAndSet" will fail and - // then that thread will update it. - if (droppedEventsCounter.compareAndSet(droppedCount, 0)) { - val prevLastReportTimestamp = lastReportTimestamp - lastReportTimestamp = System.currentTimeMillis() - val previous = new java.util.Date(prevLastReportTimestamp) - logWarning(s"Dropped $droppedCount events from $name since " + - s"${if (prevLastReportTimestamp == 0) "the application started" else s"$previous"}.") - } + val droppedEventsCount = droppedEventsCounter.get + val droppedCountIncreased = droppedEventsCount - lastDroppedEventsCounter + val lastReportTime = lastReportTimestamp.get + val curTime = System.currentTimeMillis() + // Don't log too frequently + if (droppedCountIncreased > 0 && curTime - lastReportTime >= LOGGING_INTERVAL) { + // There may be multiple threads trying to logging dropped events, + // Use 'compareAndSet' to make sure only one thread can win. + if (lastReportTimestamp.compareAndSet(lastReportTime, curTime)) { + val previous = new java.util.Date(lastReportTime) + lastDroppedEventsCounter = droppedEventsCount + logWarning(s"Dropped $droppedCountIncreased events from $name since " + + s"${if (lastReportTime == 0) "the application started" else s"$previous"}.") } } } @@ -213,4 +214,5 @@ private object AsyncEventQueue { val POISON_PILL = new SparkListenerEvent() { } + val LOGGING_INTERVAL = 60 * 1000 } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 7bf363dd71c1b..fd5c3e0827bf9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -37,7 +37,8 @@ import org.apache.spark.internal.config import org.apache.spark.internal.config.Tests.TEST_NO_STAGE_RETRY import org.apache.spark.network.util.JavaUtils import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} -import org.apache.spark.rdd.{DeterministicLevel, RDD, RDDCheckpointData} +import org.apache.spark.rdd.{RDD, RDDCheckpointData} +import org.apache.spark.resource.ResourceProfile import org.apache.spark.rpc.RpcTimeout import org.apache.spark.storage._ import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat @@ -391,7 +392,8 @@ private[spark] class DAGScheduler( val parents = getOrCreateParentStages(rdd, jobId) val id = nextStageId.getAndIncrement() val stage = new ShuffleMapStage( - id, rdd, numTasks, parents, jobId, rdd.creationSite, shuffleDep, mapOutputTracker) + id, rdd, numTasks, parents, jobId, rdd.creationSite, shuffleDep, mapOutputTracker, + ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) stageIdToStage(id) = stage shuffleIdToMapStage(shuffleDep.shuffleId) = stage @@ -453,7 +455,8 @@ private[spark] class DAGScheduler( checkBarrierStageWithRDDChainPattern(rdd, partitions.toSet.size) val parents = getOrCreateParentStages(rdd, jobId) val id = nextStageId.getAndIncrement() - val stage = new ResultStage(id, rdd, func, partitions, parents, jobId, callSite) + val stage = new ResultStage(id, rdd, func, partitions, parents, jobId, callSite, + ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) stageIdToStage(id) = stage updateJobIdStageIdMaps(jobId, stage) stage diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 8c23388b37a3d..24e2a5e4d4a62 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -40,7 +40,7 @@ import org.apache.spark.util.{JsonProtocol, Utils} * spark.eventLog.enabled - Whether event logging is enabled. * spark.eventLog.dir - Path to the directory in which events are logged. * spark.eventLog.logBlockUpdates.enabled - Whether to log block updates - * spark.eventLog.logStageExecutorMetrics.enabled - Whether to log stage executor metrics + * spark.eventLog.logStageExecutorMetrics - Whether to log stage executor metrics * * Event log file writer maintains its own parameters: refer the doc of [[EventLogFileWriter]] * and its descendant for more details. diff --git a/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala b/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala index 46a35b6a2eaf9..ee31093ec0652 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala @@ -58,3 +58,11 @@ private [spark] object LossReasonPending extends ExecutorLossReason("Pending los private[spark] case class SlaveLost(_message: String = "Slave lost", workerLost: Boolean = false) extends ExecutorLossReason(_message) + +/** + * A loss reason that means the executor is marked for decommissioning. + * + * This is used by the task scheduler to remove state associated with the executor, but + * not yet fail any tasks that were running in the executor before the executor is "fully" lost. + */ +private [spark] object ExecutorDecommission extends ExecutorLossReason("Executor decommission.") diff --git a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala index 80805df256a15..2e2851eb9070b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala @@ -88,6 +88,10 @@ private[spark] class Pool( schedulableQueue.asScala.foreach(_.executorLost(executorId, host, reason)) } + override def executorDecommission(executorId: String): Unit = { + schedulableQueue.asScala.foreach(_.executorDecommission(executorId)) + } + override def checkSpeculatableTasks(minTimeToSpeculation: Int): Boolean = { var shouldRevive = false for (schedulable <- schedulableQueue.asScala) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultStage.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultStage.scala index d1687830ff7bf..7fdc3186e86bd 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultStage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultStage.scala @@ -34,8 +34,9 @@ private[spark] class ResultStage( val partitions: Array[Int], parents: List[Stage], firstJobId: Int, - callSite: CallSite) - extends Stage(id, rdd, partitions.length, parents, firstJobId, callSite) { + callSite: CallSite, + resourceProfileId: Int) + extends Stage(id, rdd, partitions.length, parents, firstJobId, callSite, resourceProfileId) { /** * The active job for this result stage. Will be empty if the job has already finished diff --git a/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala b/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala index b6f88ed0a93aa..8cc239c81d11a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala @@ -43,6 +43,7 @@ private[spark] trait Schedulable { def removeSchedulable(schedulable: Schedulable): Unit def getSchedulableByName(name: String): Schedulable def executorLost(executorId: String, host: String, reason: ExecutorLossReason): Unit + def executorDecommission(executorId: String): Unit def checkSpeculatableTasks(minTimeToSpeculation: Int): Boolean def getSortedTaskSetQueue: ArrayBuffer[TaskSetManager] } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala index 9159d2a0158d5..4752353046c19 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala @@ -27,6 +27,9 @@ private[spark] trait SchedulerBackend { def start(): Unit def stop(): Unit + /** + * Update the current offers and schedule tasks + */ def reviveOffers(): Unit def defaultParallelism(): Int diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala index 1b44d0aee3195..be1984de9837f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapStage.scala @@ -42,8 +42,9 @@ private[spark] class ShuffleMapStage( firstJobId: Int, callSite: CallSite, val shuffleDep: ShuffleDependency[_, _, _], - mapOutputTrackerMaster: MapOutputTrackerMaster) - extends Stage(id, rdd, numTasks, parents, firstJobId, callSite) { + mapOutputTrackerMaster: MapOutputTrackerMaster, + resourceProfileId: Int) + extends Stage(id, rdd, numTasks, parents, firstJobId, callSite, resourceProfileId) { private[this] var _mapStageJobs: List[ActiveJob] = Nil diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index a9f72eae71368..ae7924d66a301 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -59,7 +59,8 @@ private[scheduler] abstract class Stage( val numTasks: Int, val parents: List[Stage], val firstJobId: Int, - val callSite: CallSite) + val callSite: CallSite, + val resourceProfileId: Int) extends Logging { val numPartitions = rdd.partitions.length @@ -79,7 +80,8 @@ private[scheduler] abstract class Stage( * StageInfo to tell SparkListeners when a job starts (which happens before any stage attempts * have been created). */ - private var _latestInfo: StageInfo = StageInfo.fromStage(this, nextAttemptId) + private var _latestInfo: StageInfo = + StageInfo.fromStage(this, nextAttemptId, resourceProfileId = resourceProfileId) /** * Set of stage attempt IDs that have failed. We keep track of these failures in order to avoid @@ -100,7 +102,8 @@ private[scheduler] abstract class Stage( val metrics = new TaskMetrics metrics.register(rdd.sparkContext) _latestInfo = StageInfo.fromStage( - this, nextAttemptId, Some(numPartitionsToCompute), metrics, taskLocalityPreferences) + this, nextAttemptId, Some(numPartitionsToCompute), metrics, taskLocalityPreferences, + resourceProfileId = resourceProfileId) nextAttemptId += 1 } diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index fdc50328b43d8..556478d83cf39 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -38,7 +38,8 @@ class StageInfo( val details: String, val taskMetrics: TaskMetrics = null, private[spark] val taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty, - private[spark] val shuffleDepId: Option[Int] = None) { + private[spark] val shuffleDepId: Option[Int] = None, + val resourceProfileId: Int) { /** When this stage was submitted from the DAGScheduler to a TaskScheduler. */ var submissionTime: Option[Long] = None /** Time when all tasks in the stage completed or when the stage was cancelled. */ @@ -87,7 +88,8 @@ private[spark] object StageInfo { attemptId: Int, numTasks: Option[Int] = None, taskMetrics: TaskMetrics = null, - taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty + taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty, + resourceProfileId: Int ): StageInfo = { val ancestorRddInfos = stage.rdd.getNarrowAncestors.map(RDDInfo.fromRdd) val rddInfos = Seq(RDDInfo.fromRdd(stage.rdd)) ++ ancestorRddInfos @@ -105,6 +107,7 @@ private[spark] object StageInfo { stage.details, taskMetrics, taskLocalityPreferences, - shuffleDepId) + shuffleDepId, + resourceProfileId) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index 15f5d20e9be75..e9e638a3645ac 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -98,6 +98,11 @@ private[spark] trait TaskScheduler { */ def applicationId(): String = appId + /** + * Process a decommissioning executor. + */ + def executorDecommission(executorId: String): Unit + /** * Process a lost executor */ diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 6a1d460e6a9d9..1b197c4cca53e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -384,7 +384,9 @@ private[spark] class TaskSchedulerImpl( */ private def resourcesMeetTaskRequirements(resources: Map[String, Buffer[String]]): Boolean = { val resourcesFree = resources.map(r => r._1 -> r._2.length) - ResourceUtils.resourcesMeetRequirements(resourcesFree, resourcesReqsPerTask) + val meetsReqs = ResourceUtils.resourcesMeetRequirements(resourcesFree, resourcesReqsPerTask) + logDebug(s"Resources meet task requirements is: $meetsReqs") + meetsReqs } /** @@ -732,6 +734,11 @@ private[spark] class TaskSchedulerImpl( } } + override def executorDecommission(executorId: String): Unit = { + rootPool.executorDecommission(executorId) + backend.reviveOffers() + } + override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = { var failedExecutor: Option[String] = None diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 2ce11347ade39..18684ee8ebbc2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -1083,6 +1083,12 @@ private[spark] class TaskSetManager( levels.toArray } + def executorDecommission(execId: String): Unit = { + recomputeLocality() + // Future consideration: if an executor is decommissioned it may make sense to add the current + // tasks to the spec exec queue. + } + def recomputeLocality(): Unit = { // A zombie TaskSetManager may reach here while executorLost happens if (isZombie) return diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index 283390814a6c0..465c0d20de481 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -94,6 +94,8 @@ private[spark] object CoarseGrainedClusterMessages { case class RemoveExecutor(executorId: String, reason: ExecutorLossReason) extends CoarseGrainedClusterMessage + case class DecommissionExecutor(executorId: String) extends CoarseGrainedClusterMessage + case class RemoveWorker(workerId: String, host: String, message: String) extends CoarseGrainedClusterMessage @@ -115,9 +117,9 @@ private[spark] object CoarseGrainedClusterMessages { // Request executors by specifying the new total number of executors desired // This includes executors already pending or running case class RequestExecutors( - requestedTotal: Int, - localityAwareTasks: Int, - hostToLocalTaskCount: Map[String, Int], + resourceProfileToTotalExecs: Map[ResourceProfile, Int], + numLocalityAwareTasksPerResourceProfileId: Map[Int, Int], + hostToLocalTaskCount: Map[Int, Map[String, Int]], nodeBlacklist: Set[String]) extends CoarseGrainedClusterMessage diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 55f4005ef1b45..6e1efdaf5beb2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -69,13 +69,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp conf.get(SCHEDULER_MAX_REGISTERED_RESOURCE_WAITING_TIME)) private val createTimeNs = System.nanoTime() - private val taskResourceNumParts: Map[String, Int] = - if (scheduler.resourcesReqsPerTask != null) { - scheduler.resourcesReqsPerTask.map(req => req.resourceName -> req.numParts).toMap - } else { - Map.empty - } - // Accessing `executorDataMap` in the inherited methods from ThreadSafeRpcEndpoint doesn't need // any protection. But accessing `executorDataMap` out of the inherited methods must be // protected by `CoarseGrainedSchedulerBackend.this`. Besides, `executorDataMap` should only @@ -83,13 +76,10 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // `CoarseGrainedSchedulerBackend.this`. private val executorDataMap = new HashMap[String, ExecutorData] - // Number of executors requested by the cluster manager, [[ExecutorAllocationManager]] - @GuardedBy("CoarseGrainedSchedulerBackend.this") - private var requestedTotalExecutors = 0 - - // Number of executors requested from the cluster manager that have not registered yet + // Number of executors for each ResourceProfile requested by the cluster + // manager, [[ExecutorAllocationManager]] @GuardedBy("CoarseGrainedSchedulerBackend.this") - private var numPendingExecutors = 0 + private val requestedTotalExecutorsPerResourceProfile = new HashMap[ResourceProfile, Int] private val listenerBus = scheduler.sc.listenerBus @@ -102,13 +92,16 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // Executors that have been lost, but for which we don't yet know the real exit reason. private val executorsPendingLossReason = new HashSet[String] - // A map to store hostname with its possible task number running on it + // Executors which are being decommissioned + protected val executorsPendingDecommission = new HashSet[String] + + // A map of ResourceProfile id to map of hostname with its possible task number running on it @GuardedBy("CoarseGrainedSchedulerBackend.this") - protected var hostToLocalTaskCount: Map[String, Int] = Map.empty + protected var rpHostToLocalTaskCount: Map[Int, Map[String, Int]] = Map.empty - // The number of pending tasks which is locality required + // The number of pending tasks per ResourceProfile id which is locality required @GuardedBy("CoarseGrainedSchedulerBackend.this") - protected var localityAwareTasks = 0 + protected var numLocalityAwareTasksPerResourceProfileId = Map.empty[Int, Int] // The num of current max ExecutorId used to re-register appMaster @volatile protected var currentExecutorIdCounter = 0 @@ -195,11 +188,20 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp executorDataMap.get(executorId).foreach(_.executorEndpoint.send(StopExecutor)) removeExecutor(executorId, reason) + case DecommissionExecutor(executorId) => + logError(s"Received decommission executor message ${executorId}.") + decommissionExecutor(executorId) + + case RemoveWorker(workerId, host, message) => + removeWorker(workerId, host, message) + case LaunchedExecutor(executorId) => executorDataMap.get(executorId).foreach { data => data.freeCores = data.totalCores } makeOffers(executorId) + case e => + logError(s"Received unexpected message. ${e}") } override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { @@ -223,16 +225,17 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp } else { context.senderAddress } - logInfo(s"Registered executor $executorRef ($executorAddress) with ID $executorId") + logInfo(s"Registered executor $executorRef ($executorAddress) with ID $executorId, " + + s" ResourceProfileId $resourceProfileId") addressToExecutorId(executorAddress) = executorId totalCoreCount.addAndGet(cores) totalRegisteredExecutors.addAndGet(1) - val resourcesInfo = resources.map{ case (k, v) => - (v.name, - new ExecutorResourceInfo(v.name, v.addresses, - // tell the executor it can schedule resources up to numParts times, - // as configured by the user, or set to 1 as that is the default (1 task/resource) - taskResourceNumParts.getOrElse(v.name, 1))) + val resourcesInfo = resources.map { case (rName, info) => + // tell the executor it can schedule resources up to numParts times, + // as configured by the user, or set to 1 as that is the default (1 task/resource) + val numParts = scheduler.sc.resourceProfileManager + .resourceProfileFromId(resourceProfileId).getNumSlotsPerAddress(rName, conf) + (info.name, new ExecutorResourceInfo(info.name, info.addresses, numParts)) } val data = new ExecutorData(executorRef, executorAddress, hostname, 0, cores, logUrlHandler.applyPattern(logUrls, attributes), attributes, @@ -244,10 +247,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp if (currentExecutorIdCounter < executorId.toInt) { currentExecutorIdCounter = executorId.toInt } - if (numPendingExecutors > 0) { - numPendingExecutors -= 1 - logDebug(s"Decremented number of pending executors ($numPendingExecutors left)") - } } // Note: some tests expect the reply to come after we put the executor in the map context.reply(true) @@ -270,17 +269,21 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp removeWorker(workerId, host, message) context.reply(true) + case DecommissionExecutor(executorId) => + logError(s"Received decommission executor message ${executorId}.") + decommissionExecutor(executorId) + context.reply(true) + case RetrieveSparkAppConfig(resourceProfileId) => - // note this will be updated in later prs to get the ResourceProfile from a - // ResourceProfileManager that matches the resource profile id - // for now just use default profile - val rp = ResourceProfile.getOrCreateDefaultProfile(conf) + val rp = scheduler.sc.resourceProfileManager.resourceProfileFromId(resourceProfileId) val reply = SparkAppConfig( sparkProperties, SparkEnv.get.securityManager.getIOEncryptionKey(), Option(delegationTokens.get()), rp) context.reply(reply) + case e => + logError(s"Received unexpected ask ${e}") } // Make fake resource offers on all executors @@ -381,6 +384,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp addressToExecutorId -= executorInfo.executorAddress executorDataMap -= executorId executorsPendingLossReason -= executorId + executorsPendingDecommission -= executorId executorsPendingToRemove.remove(executorId).getOrElse(false) } totalCoreCount.addAndGet(-executorInfo.totalCores) @@ -405,6 +409,35 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp scheduler.workerRemoved(workerId, host, message) } + /** + * Mark a given executor as decommissioned and stop making resource offers for it. + */ + private def decommissionExecutor(executorId: String): Boolean = { + val shouldDisable = CoarseGrainedSchedulerBackend.this.synchronized { + // Only bother decommissioning executors which are alive. + if (isExecutorActive(executorId)) { + executorsPendingDecommission += executorId + true + } else { + false + } + } + + if (shouldDisable) { + logInfo(s"Starting decommissioning executor $executorId.") + try { + scheduler.executorDecommission(executorId) + } catch { + case e: Exception => + logError(s"Unexpected error during decommissioning ${e.toString}", e) + } + logInfo(s"Finished decommissioning executor $executorId.") + } else { + logInfo(s"Skipping decommissioning of executor $executorId.") + } + shouldDisable + } + /** * Stop making resource offers for the given executor. The executor is marked as lost with * the loss reason still pending. @@ -494,8 +527,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp * */ protected[scheduler] def reset(): Unit = { val executors: Set[String] = synchronized { - requestedTotalExecutors = 0 - numPendingExecutors = 0 + requestedTotalExecutorsPerResourceProfile.clear() executorDataMap.keys.toSet } @@ -528,8 +560,17 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp } protected def removeWorker(workerId: String, host: String, message: String): Unit = { - driverEndpoint.ask[Boolean](RemoveWorker(workerId, host, message)).failed.foreach(t => - logError(t.getMessage, t))(ThreadUtils.sameThread) + driverEndpoint.send(RemoveWorker(workerId, host, message)) + } + + /** + * Called by subclasses when notified of a decommissioning executor. + */ + private[spark] def decommissionExecutor(executorId: String): Unit = { + if (driverEndpoint != null) { + logInfo("Propegating executor decommission to driver.") + driverEndpoint.send(DecommissionExecutor(executorId)) + } } def sufficientResourcesRegistered(): Boolean = true @@ -560,7 +601,9 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp override def isExecutorActive(id: String): Boolean = synchronized { executorDataMap.contains(id) && !executorsPendingToRemove.contains(id) && - !executorsPendingLossReason.contains(id) + !executorsPendingLossReason.contains(id) && + !executorsPendingDecommission.contains(id) + } override def maxNumConcurrentTasks(): Int = synchronized { @@ -577,12 +620,14 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // this function is for testing only def getExecutorResourceProfileId(executorId: String): Int = synchronized { - val res = executorDataMap.get(executorId) - res.map(_.resourceProfileId).getOrElse(ResourceProfile.UNKNOWN_RESOURCE_PROFILE_ID) + val execDataOption = executorDataMap.get(executorId) + execDataOption.map(_.resourceProfileId).getOrElse(ResourceProfile.UNKNOWN_RESOURCE_PROFILE_ID) } /** - * Request an additional number of executors from the cluster manager. + * Request an additional number of executors from the cluster manager. This is + * requesting against the default ResourceProfile, we will need an API change to + * allow against other profiles. * @return whether the request is acknowledged. */ final override def requestExecutors(numAdditionalExecutors: Int): Boolean = { @@ -594,21 +639,11 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp logInfo(s"Requesting $numAdditionalExecutors additional executor(s) from the cluster manager") val response = synchronized { - requestedTotalExecutors += numAdditionalExecutors - numPendingExecutors += numAdditionalExecutors - logDebug(s"Number of pending executors is now $numPendingExecutors") - if (requestedTotalExecutors != - (numExistingExecutors + numPendingExecutors - executorsPendingToRemove.size)) { - logDebug( - s"""requestExecutors($numAdditionalExecutors): Executor request doesn't match: - |requestedTotalExecutors = $requestedTotalExecutors - |numExistingExecutors = $numExistingExecutors - |numPendingExecutors = $numPendingExecutors - |executorsPendingToRemove = ${executorsPendingToRemove.size}""".stripMargin) - } - + val defaultProf = scheduler.sc.resourceProfileManager.defaultResourceProfile + val numExisting = requestedTotalExecutorsPerResourceProfile.getOrElse(defaultProf, 0) + requestedTotalExecutorsPerResourceProfile(defaultProf) = numExisting + numAdditionalExecutors // Account for executors pending to be added or removed - doRequestTotalExecutors(requestedTotalExecutors) + doRequestTotalExecutors(requestedTotalExecutorsPerResourceProfile.toMap) } defaultAskTimeout.awaitResult(response) @@ -617,39 +652,41 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp /** * Update the cluster manager on our scheduling needs. Three bits of information are included * to help it make decisions. - * @param numExecutors The total number of executors we'd like to have. The cluster manager - * shouldn't kill any running executor to reach this number, but, - * if all existing executors were to die, this is the number of executors - * we'd want to be allocated. - * @param localityAwareTasks The number of tasks in all active stages that have a locality - * preferences. This includes running, pending, and completed tasks. + * @param resourceProfileToNumExecutors The total number of executors we'd like to have per + * ResourceProfile. The cluster manager shouldn't kill any + * running executor to reach this number, but, if all + * existing executors were to die, this is the number + * of executors we'd want to be allocated. + * @param numLocalityAwareTasksPerResourceProfileId The number of tasks in all active stages that + * have a locality preferences per + * ResourceProfile. This includes running, + * pending, and completed tasks. * @param hostToLocalTaskCount A map of hosts to the number of tasks from all active stages * that would like to like to run on that host. * This includes running, pending, and completed tasks. * @return whether the request is acknowledged by the cluster manager. */ final override def requestTotalExecutors( - numExecutors: Int, - localityAwareTasks: Int, - hostToLocalTaskCount: Map[String, Int] - ): Boolean = { - if (numExecutors < 0) { + resourceProfileIdToNumExecutors: Map[Int, Int], + numLocalityAwareTasksPerResourceProfileId: Map[Int, Int], + hostToLocalTaskCount: Map[Int, Map[String, Int]] + ): Boolean = { + val totalExecs = resourceProfileIdToNumExecutors.values.sum + if (totalExecs < 0) { throw new IllegalArgumentException( "Attempted to request a negative number of executor(s) " + - s"$numExecutors from the cluster manager. Please specify a positive number!") + s"$totalExecs from the cluster manager. Please specify a positive number!") + } + val resourceProfileToNumExecutors = resourceProfileIdToNumExecutors.map { case (rpid, num) => + (scheduler.sc.resourceProfileManager.resourceProfileFromId(rpid), num) } - val response = synchronized { - this.requestedTotalExecutors = numExecutors - this.localityAwareTasks = localityAwareTasks - this.hostToLocalTaskCount = hostToLocalTaskCount - - numPendingExecutors = - math.max(numExecutors - numExistingExecutors + executorsPendingToRemove.size, 0) - - doRequestTotalExecutors(numExecutors) + this.requestedTotalExecutorsPerResourceProfile.clear() + this.requestedTotalExecutorsPerResourceProfile ++= resourceProfileToNumExecutors + this.numLocalityAwareTasksPerResourceProfileId = numLocalityAwareTasksPerResourceProfileId + this.rpHostToLocalTaskCount = hostToLocalTaskCount + doRequestTotalExecutors(requestedTotalExecutorsPerResourceProfile.toMap) } - defaultAskTimeout.awaitResult(response) } @@ -665,7 +702,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp * * @return a future whose evaluation indicates whether the request is acknowledged. */ - protected def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = + protected def doRequestTotalExecutors( + resourceProfileToTotalExecs: Map[ResourceProfile, Int]): Future[Boolean] = Future.successful(false) /** @@ -706,20 +744,20 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // take into account executors that are pending to be added or removed. val adjustTotalExecutors = if (adjustTargetNumExecutors) { - requestedTotalExecutors = math.max(requestedTotalExecutors - executorsToKill.size, 0) - if (requestedTotalExecutors != - (numExistingExecutors + numPendingExecutors - executorsPendingToRemove.size)) { - logDebug( - s"""killExecutors($executorIds, $adjustTargetNumExecutors, $countFailures, $force): - |Executor counts do not match: - |requestedTotalExecutors = $requestedTotalExecutors - |numExistingExecutors = $numExistingExecutors - |numPendingExecutors = $numPendingExecutors - |executorsPendingToRemove = ${executorsPendingToRemove.size}""".stripMargin) + executorsToKill.foreach { exec => + val rpId = executorDataMap(exec).resourceProfileId + val rp = scheduler.sc.resourceProfileManager.resourceProfileFromId(rpId) + if (requestedTotalExecutorsPerResourceProfile.isEmpty) { + // Assume that we are killing an executor that was started by default and + // not through the request api + requestedTotalExecutorsPerResourceProfile(rp) = 0 + } else { + val requestedTotalForRp = requestedTotalExecutorsPerResourceProfile(rp) + requestedTotalExecutorsPerResourceProfile(rp) = math.max(requestedTotalForRp - 1, 0) + } } - doRequestTotalExecutors(requestedTotalExecutors) + doRequestTotalExecutors(requestedTotalExecutorsPerResourceProfile.toMap) } else { - numPendingExecutors += executorsToKill.size Future.successful(true) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index a9b607d8cc38c..42c46464d79e1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -28,7 +28,7 @@ import org.apache.spark.deploy.client.{StandaloneAppClient, StandaloneAppClientL import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.config.Tests.IS_TESTING import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle} -import org.apache.spark.resource.ResourceUtils +import org.apache.spark.resource.{ResourceProfile, ResourceUtils} import org.apache.spark.rpc.RpcEndpointAddress import org.apache.spark.scheduler._ import org.apache.spark.util.Utils @@ -58,6 +58,7 @@ private[spark] class StandaloneSchedulerBackend( private val maxCores = conf.get(config.CORES_MAX) private val totalExpectedCores = maxCores.getOrElse(0) + private val defaultProf = sc.resourceProfileManager.defaultResourceProfile override def start(): Unit = { super.start() @@ -173,6 +174,12 @@ private[spark] class StandaloneSchedulerBackend( removeExecutor(fullId.split("/")(1), reason) } + override def executorDecommissioned(fullId: String, message: String) { + logInfo("Asked to decommission executor") + decommissionExecutor(fullId.split("/")(1)) + logInfo("Executor %s decommissioned: %s".format(fullId, message)) + } + override def workerRemoved(workerId: String, host: String, message: String): Unit = { logInfo("Worker %s removed: %s".format(workerId, message)) removeWorker(workerId, host, message) @@ -194,9 +201,13 @@ private[spark] class StandaloneSchedulerBackend( * * @return whether the request is acknowledged. */ - protected override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = { + protected override def doRequestTotalExecutors( + resourceProfileToTotalExecs: Map[ResourceProfile, Int]): Future[Boolean] = { + // resources profiles not supported Option(client) match { - case Some(c) => c.requestTotalExecutors(requestedTotal) + case Some(c) => + val numExecs = resourceProfileToTotalExecs.getOrElse(defaultProf, 0) + c.requestTotalExecutors(numExecs) case None => logWarning("Attempted to request executors before driver fully initialized.") Future.successful(false) diff --git a/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala b/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala index a24f1902faa31..c29546b7577fc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala @@ -70,7 +70,7 @@ private[spark] class ExecutorMonitor( // this listener. There are safeguards in other parts of the code that would prevent that executor // from being removed. private val nextTimeout = new AtomicLong(Long.MaxValue) - private var timedOutExecs = Seq.empty[String] + private var timedOutExecs = Seq.empty[(String, Int)] // Active job tracking. // @@ -100,10 +100,10 @@ private[spark] class ExecutorMonitor( } /** - * Returns the list of executors that are currently considered to be timed out. - * Should only be called from the EAM thread. + * Returns the list of executors and their ResourceProfile id that are currently considered to + * be timed out. Should only be called from the EAM thread. */ - def timedOutExecutors(): Seq[String] = { + def timedOutExecutors(): Seq[(String, Int)] = { val now = clock.nanoTime() if (now >= nextTimeout.get()) { // Temporarily set the next timeout at Long.MaxValue. This ensures that after @@ -126,7 +126,7 @@ private[spark] class ExecutorMonitor( true } } - .keys + .map { case (name, exec) => (name, exec.resourceProfileId)} .toSeq updateNextTimeout(newNextTimeout) } @@ -155,6 +155,7 @@ private[spark] class ExecutorMonitor( execResourceProfileCount.getOrDefault(id, 0) } + // for testing def getResourceProfileId(executorId: String): Int = { val execTrackingInfo = executors.get(executorId) if (execTrackingInfo != null) { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index c47901314f53a..e7f8de5ab7e4a 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -666,7 +666,11 @@ private[spark] class BlockManager( // stream. channel.close() val blockSize = channel.getCount - TempFileBasedBlockStoreUpdater(blockId, level, classTag, tmpFile, blockSize).save() + val blockStored = TempFileBasedBlockStoreUpdater( + blockId, level, classTag, tmpFile, blockSize).save() + if (!blockStored) { + throw new Exception(s"Failure while trying to store block $blockId on $blockManagerId.") + } } override def onFailure(streamId: String, cause: Throwable): Unit = { diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 4d89c4f079f29..9254ac94005f1 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -22,7 +22,7 @@ import java.util.{Properties, UUID} import scala.collection.JavaConverters._ import scala.collection.Map -import com.fasterxml.jackson.databind.ObjectMapper +import com.fasterxml.jackson.databind.{DeserializationFeature, ObjectMapper} import com.fasterxml.jackson.module.scala.DefaultScalaModule import org.json4s.DefaultFormats import org.json4s.JsonAST._ @@ -33,7 +33,7 @@ import org.apache.spark._ import org.apache.spark.executor._ import org.apache.spark.metrics.ExecutorMetricType import org.apache.spark.rdd.RDDOperationScope -import org.apache.spark.resource.ResourceInformation +import org.apache.spark.resource.{ResourceInformation, ResourceProfile} import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.storage._ @@ -59,6 +59,7 @@ private[spark] object JsonProtocol { private implicit val format = DefaultFormats private val mapper = new ObjectMapper().registerModule(DefaultScalaModule) + .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false) /** ------------------------------------------------- * * JSON serialization methods for SparkListenerEvents | @@ -662,7 +663,8 @@ private[spark] object JsonProtocol { val stageInfos = jsonOption(json \ "Stage Infos") .map(_.extract[Seq[JValue]].map(stageInfoFromJson)).getOrElse { stageIds.map { id => - new StageInfo(id, 0, "unknown", 0, Seq.empty, Seq.empty, "unknown") + new StageInfo(id, 0, "unknown", 0, Seq.empty, Seq.empty, "unknown", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) } } SparkListenerJobStart(jobId, submissionTime, stageInfos, properties) @@ -803,7 +805,8 @@ private[spark] object JsonProtocol { } val stageInfo = new StageInfo( - stageId, attemptId, stageName, numTasks, rddInfos, parentIds, details) + stageId, attemptId, stageName, numTasks, rddInfos, parentIds, details, + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) stageInfo.submissionTime = submissionTime stageInfo.completionTime = completionTime stageInfo.failureReason = failureReason diff --git a/core/src/main/scala/org/apache/spark/util/SignalUtils.scala b/core/src/main/scala/org/apache/spark/util/SignalUtils.scala index 5a24965170cef..230195da2a121 100644 --- a/core/src/main/scala/org/apache/spark/util/SignalUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/SignalUtils.scala @@ -60,7 +60,7 @@ private[spark] object SignalUtils extends Logging { if (SystemUtils.IS_OS_UNIX) { try { val handler = handlers.getOrElseUpdate(signal, { - logInfo("Registered signal handler for " + signal) + logInfo("Registering signal handler for " + signal) new ActionHandler(new Signal(signal)) }) handler.register(action) diff --git a/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala b/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala index de39e4b410f25..e7872bb9cb6b0 100644 --- a/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala @@ -18,6 +18,7 @@ package org.apache.spark.util import java.util.concurrent._ +import java.util.concurrent.{Future => JFuture} import java.util.concurrent.locks.ReentrantLock import scala.concurrent.{Awaitable, ExecutionContext, ExecutionContextExecutor, Future} @@ -304,6 +305,22 @@ private[spark] object ThreadUtils { } // scalastyle:on awaitresult + @throws(classOf[SparkException]) + def awaitResult[T](future: JFuture[T], atMost: Duration): T = { + try { + atMost match { + case Duration.Inf => future.get() + case _ => future.get(atMost._1, atMost._2) + } + } catch { + case e: SparkFatalException => + throw e.throwable + case NonFatal(t) + if !t.isInstanceOf[TimeoutException] && !t.isInstanceOf[RpcAbortException] => + throw new SparkException("Exception thrown in awaitResult: ", t) + } + } + // scalastyle:off awaitready /** * Preferred alternative to `Await.ready()`. diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 297cc5e4cb100..dde43232f0d65 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -2772,19 +2772,16 @@ private[spark] object Utils extends Logging { } val masterScheme = new URI(masterWithoutK8sPrefix).getScheme - val resolvedURL = masterScheme.toLowerCase(Locale.ROOT) match { - case "https" => + + val resolvedURL = Option(masterScheme).map(_.toLowerCase(Locale.ROOT)) match { + case Some("https") => masterWithoutK8sPrefix - case "http" => + case Some("http") => logWarning("Kubernetes master URL uses HTTP instead of HTTPS.") masterWithoutK8sPrefix - case null => - val resolvedURL = s"https://$masterWithoutK8sPrefix" - logInfo("No scheme specified for kubernetes master URL, so defaulting to https. Resolved " + - s"URL is $resolvedURL.") - resolvedURL case _ => - throw new IllegalArgumentException("Invalid Kubernetes master scheme: " + masterScheme) + throw new IllegalArgumentException("Invalid Kubernetes master scheme: " + masterScheme + + " found in URL: " + masterWithoutK8sPrefix) } s"k8s://$resolvedURL" diff --git a/core/src/test/resources/spark-events/application_1553914137147_0018 b/core/src/test/resources/spark-events/application_1553914137147_0018 index 8c34e7265f8da..03ea3040de1db 100644 --- a/core/src/test/resources/spark-events/application_1553914137147_0018 +++ b/core/src/test/resources/spark-events/application_1553914137147_0018 @@ -1,6 +1,6 @@ {"Event":"SparkListenerLogStart","Spark Version":"3.0.0-SNAPSHOT"} {"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"test-1.vpc.company.com","Port":44768},"Maximum Memory":956615884,"Timestamp":1554755989747,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0} -{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/usr/java/jdk1.8.0_144/jre","Java Version":"1.8.0_144 (Oracle Corporation)","Scala Version":"version 2.12.8"},"Spark Properties":{"spark.lineage.log.dir":"/var/log/spark2/lineage","spark.serializer":"org.apache.spark.serializer.KryoSerializer","spark.yarn.jars":"local:/opt/cloudera/parcels/SPARK2/lib/spark2/jars/*","spark.executor.extraJavaOptions":"-Djava.security.egd=file:///dev/urandom","spark.driver.host":"test-1.vpc.company.com","spark.eventLog.enabled":"true","spark.executor.heartbeatInterval":"1000","spark.executor.memoryOverhead":"1024","spark.driver.port":"34194","spark.shuffle.service.enabled":"false","spark.driver.extraLibraryPath":"/opt/cloudera/parcels/CDH/lib/hadoop/lib/native","spark.lineage.enabled":"false","spark.jars":"file:/tmp/__spark_test__/spark3-tests-0.1.0-cdh5.9.0-SNAPSHOT-jar-with-dependencies.jar","spark.executor.metrics.pollingInterval":"100","spark.yarn.historyServer.address":"http://test-1.vpc.company.com:18089","spark.ui.enabled":"true","spark.app.name":"LargeBlocks","spark.ui.killEnabled":"true","spark.sql.hive.metastore.jars":"${env:HADOOP_COMMON_HOME}/../hive/lib/*:${env:HADOOP_COMMON_HOME}/client/*","spark.locality.wait.process":"0","spark.dynamicAllocation.schedulerBacklogTimeout":"1","spark.yarn.am.extraLibraryPath":"/opt/cloudera/parcels/CDH/lib/hadoop/lib/native","spark.scheduler.mode":"FIFO","spark.eventLog.logStageExecutorMetrics.enabled":"true","spark.driver.memory":"2g","spark.executor.instances":"3","spark.submit.pyFiles":"","spark.yarn.config.gatewayPath":"/opt/cloudera/parcels","spark.executor.id":"driver","spark.yarn.config.replacementPath":"{{HADOOP_COMMON_HOME}}/../../..","spark.driver.extraJavaOptions":"-Djava.security.egd=file:///dev/urandom","spark.eventLog.logStageExecutorProcessTreeMetrics.enabled":"true","spark.submit.deployMode":"client","spark.shuffle.service.port":"7337","spark.yarn.maxAppAttempts":"1","spark.master":"yarn","spark.authenticate":"false","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.network.crypto.enabled":"false","spark.executor.extraLibraryPath":"/opt/cloudera/parcels/CDH/lib/hadoop/lib/native","spark.executor.memory":"7g","spark.io.encryption.enabled":"false","spark.eventLog.dir":"hdfs://test-1.vpc.company.com:8020/user/spark/spark2ApplicationHistory","spark.dynamicAllocation.enabled":"false","spark.sql.catalogImplementation":"hive","spark.executor.cores":"1","spark.driver.appUIAddress":"http://test-1.vpc.company.com:4040","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"test-1.vpc.company.com","spark.dynamicAllocation.minExecutors":"0","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://test-1.vpc.company.com:8088/proxy/application_1553914137147_0018","spark.dynamicAllocation.executorIdleTimeout":"60","spark.app.id":"application_1553914137147_0018","spark.sql.hive.metastore.version":"1.1.0"},"Hadoop Properties":{"yarn.resourcemanager.amlauncher.thread-count":"50","dfs.namenode.resource.check.interval":"5000","fs.s3a.connection.maximum":"100","mapreduce.jobtracker.jobhistory.task.numberprogresssplits":"12","dfs.data.transfer.server.tcpnodelay":"true","mapreduce.tasktracker.healthchecker.script.timeout":"600000","fs.s3a.impl":"org.apache.hadoop.fs.s3a.S3AFileSystem","yarn.app.mapreduce.am.scheduler.heartbeat.interval-ms":"1000","hadoop.security.kms.client.timeout":"60","hadoop.http.authentication.kerberos.principal":"HTTP/_HOST@LOCALHOST","mapreduce.jobhistory.loadedjob.tasks.max":"-1","mapreduce.framework.name":"yarn","yarn.nodemanager.linux-container-executor.nonsecure-mode.user-pattern":"^[_.A-Za-z0-9][-@_.A-Za-z0-9]{0,255}?[$]?$","dfs.cachereport.intervalMsec":"10000","dfs.namenode.checkpoint.txns":"1000000","tfile.fs.output.buffer.size":"262144","yarn.app.mapreduce.am.job.task.listener.thread-count":"30","mapreduce.tasktracker.local.dir.minspacekill":"0","hadoop.security.groups.cache.background.reload.threads":"3","dfs.namenode.lease-recheck-interval-ms":"2000","fs.s3.block.size":"67108864","dfs.client.domain.socket.data.traffic":"false","dfs.ha.zkfc.nn.http.timeout.ms":"20000","hadoop.registry.secure":"false","hadoop.hdfs.configuration.version":"1","dfs.bytes-per-checksum":"512","fs.s3.buffer.dir":"${hadoop.tmp.dir}/s3","mapreduce.job.acl-view-job":" ","fs.s3a.s3guard.ddb.background.sleep":"25","mapreduce.jobhistory.loadedjobs.cache.size":"5","mapreduce.jobtracker.persist.jobstatus.hours":"1","fs.s3a.s3guard.ddb.table.create":"false","dfs.datanode.slow.io.warning.threshold.ms":"300","dfs.namenode.handler.count":"10","dfs.namenode.list.reencryption.status.num.responses":"100","mapreduce.input.fileinputformat.split.minsize":"0","dfs.datanode.failed.volumes.tolerated":"0","yarn.resourcemanager.container.liveness-monitor.interval-ms":"600000","yarn.resourcemanager.amliveliness-monitor.interval-ms":"1000","yarn.resourcemanager.client.thread-count":"50","io.seqfile.compress.blocksize":"1000000","mapreduce.tasktracker.http.threads":"40","dfs.namenode.retrycache.expirytime.millis":"600000","dfs.namenode.backup.address":"0.0.0.0:50100","dfs.datanode.data.dir":"file://${hadoop.tmp.dir}/dfs/data","dfs.datanode.shared.file.descriptor.paths":"/dev/shm,/tmp","dfs.replication":"3","mapreduce.jobtracker.jobhistory.block.size":"3145728","dfs.encrypt.data.transfer.cipher.key.bitlength":"128","mapreduce.reduce.shuffle.fetch.retry.interval-ms":"1000","dfs.secondary.namenode.kerberos.internal.spnego.principal":"${dfs.web.authentication.kerberos.principal}","mapreduce.task.profile.maps":"0-2","dfs.datanode.block-pinning.enabled":"false","yarn.nodemanager.admin-env":"MALLOC_ARENA_MAX=$MALLOC_ARENA_MAX","mapreduce.jobtracker.retiredjobs.cache.size":"1000","mapreduce.am.max-attempts":"2","hadoop.security.kms.client.failover.sleep.base.millis":"100","mapreduce.jobhistory.webapp.https.address":"test-1.vpc.company.com:19890","fs.trash.checkpoint.interval":"0","dfs.namenode.checkpoint.check.period":"60","yarn.nodemanager.container-monitor.interval-ms":"3000","mapreduce.job.map.output.collector.class":"org.apache.hadoop.mapred.MapTask$MapOutputBuffer","hadoop.http.authentication.signature.secret.file":"*********(redacted)","hadoop.jetty.logs.serve.aliases":"true","hadoop.proxyuser.HTTP.groups":"*","yarn.timeline-service.handler-thread-count":"10","yarn.resourcemanager.max-completed-applications":"10000","dfs.namenode.reencrypt.edek.threads":"10","yarn.resourcemanager.system-metrics-publisher.enabled":"false","hadoop.security.groups.negative-cache.secs":"30","yarn.app.mapreduce.task.container.log.backups":"0","hadoop.security.group.mapping.ldap.posix.attr.gid.name":"gidNumber","ipc.client.fallback-to-simple-auth-allowed":"false","dfs.namenode.fs-limits.max-component-length":"255","mapreduce.tasktracker.taskcontroller":"org.apache.hadoop.mapred.DefaultTaskController","yarn.client.failover-proxy-provider":"org.apache.hadoop.yarn.client.ConfiguredRMFailoverProxyProvider","yarn.timeline-service.http-authentication.simple.anonymous.allowed":"true","ha.health-monitor.check-interval.ms":"1000","dfs.namenode.top.window.num.buckets":"10","yarn.resourcemanager.store.class":"org.apache.hadoop.yarn.server.resourcemanager.recovery.FileSystemRMStateStore","dfs.datanode.block.id.layout.upgrade.threads":"12","mapreduce.jobtracker.tasktracker.maxblacklists":"4","yarn.nodemanager.docker-container-executor.exec-name":"/usr/bin/docker","yarn.resourcemanager.nodemanagers.heartbeat-interval-ms":"1000","hadoop.common.configuration.version":"0.23.0","fs.s3a.s3guard.ddb.table.capacity.read":"500","yarn.nodemanager.remote-app-log-dir-suffix":"logs","dfs.namenode.decommission.max.concurrent.tracked.nodes":"100","file.blocksize":"67108864","hadoop.registry.zk.retry.ceiling.ms":"60000","mapreduce.jobhistory.principal":"jhs/_HOST@REALM.TLD","dfs.client.read.shortcircuit.skip.checksum":"false","mapreduce.task.profile.reduces":"0-2","dfs.datanode.address":"0.0.0.0:50010","dfs.https.server.keystore.resource":"ssl-server.xml","yarn.timeline-service.webapp.https.address":"${yarn.timeline-service.hostname}:8190","yarn.resourcemanager.scheduler.address":"test-1.vpc.company.com:8030","mapreduce.task.timeout":"600000","hadoop.security.crypto.cipher.suite":"AES/CTR/NoPadding","yarn.resourcemanager.connect.max-wait.ms":"900000","fs.defaultFS":"hdfs://test-1.vpc.company.com:8020","fs.har.impl.disable.cache":"true","io.compression.codec.bzip2.library":"system-native","dfs.namenode.audit.loggers":"default","dfs.block.access.key.update.interval":"600","mapreduce.shuffle.connection-keep-alive.timeout":"5","yarn.resourcemanager.webapp.https.address":"test-1.vpc.company.com:8090","dfs.namenode.max.objects":"0","mapreduce.jobhistory.address":"test-1.vpc.company.com:10020","yarn.nodemanager.address":"${yarn.nodemanager.hostname}:0","fs.AbstractFileSystem.s3a.impl":"org.apache.hadoop.fs.s3a.S3A","mapreduce.task.combine.progress.records":"10000","dfs.namenode.max.full.block.report.leases":"6","yarn.resourcemanager.am.max-attempts":"2","yarn.nodemanager.linux-container-executor.cgroups.hierarchy":"/hadoop-yarn","dfs.client.mmap.cache.timeout.ms":"3600000","dfs.mover.max-no-move-interval":"60000","fs.ftp.transfer.mode":"BLOCK_TRANSFER_MODE","dfs.client.datanode-restart.timeout":"30","dfs.datanode.drop.cache.behind.reads":"false","ipc.server.log.slow.rpc":"false","dfs.namenode.read-lock-reporting-threshold-ms":"5000","yarn.app.mapreduce.am.job.committer.cancel-timeout":"60000","yarn.nodemanager.default-container-executor.log-dirs.permissions":"710","dfs.namenode.checkpoint.edits.dir":"${dfs.namenode.checkpoint.dir}","yarn.app.attempt.diagnostics.limit.kc":"64","dfs.balancer.block-move.timeout":"0","dfs.client.block.write.replace-datanode-on-failure.enable":"true","ftp.bytes-per-checksum":"512","yarn.nodemanager.resource.memory-mb":"8192","io.compression.codecs":"org.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.BZip2Codec,org.apache.hadoop.io.compress.DeflateCodec,org.apache.hadoop.io.compress.SnappyCodec,org.apache.hadoop.io.compress.Lz4Codec","fs.s3a.fast.upload.active.blocks":"4","hadoop.security.credential.clear-text-fallback":"true","dfs.heartbeat.interval":"3","mapreduce.jobhistory.joblist.cache.size":"20000","fs.ftp.host":"0.0.0.0","dfs.ha.tail-edits.period":"60","dfs.datanode.max.locked.memory":"0","dfs.datanode.scan.period.hours":"504","mapreduce.jobtracker.expire.trackers.interval":"600000","yarn.resourcemanager.nodemanager-connect-retries":"10","hadoop.security.kms.client.encrypted.key.cache.low-watermark":"0.3f","yarn.timeline-service.client.max-retries":"30","dfs.ha.fencing.ssh.connect-timeout":"30000","yarn.log-aggregation-enable":"false","mapreduce.reduce.markreset.buffer.percent":"0.0","fs.AbstractFileSystem.viewfs.impl":"org.apache.hadoop.fs.viewfs.ViewFs","dfs.namenode.edits.noeditlogchannelflush":"false","mapreduce.task.io.sort.factor":"64","mapreduce.tasktracker.outofband.heartbeat":"false","ha.failover-controller.new-active.rpc-timeout.ms":"60000","dfs.webhdfs.ugi.expire.after.access":"600000","mapreduce.jobhistory.datestring.cache.size":"200000","mapreduce.job.acl-modify-job":" ","dfs.namenode.https-address":"test-1.vpc.company.com:20102","yarn.am.blacklisting.enabled":"true","yarn.timeline-service.webapp.address":"${yarn.timeline-service.hostname}:8188","dfs.image.transfer-bootstrap-standby.bandwidthPerSec":"0","yarn.app.mapreduce.am.job.committer.commit-window":"10000","yarn.nodemanager.container-manager.thread-count":"20","yarn.timeline-service.ttl-enable":"true","mapreduce.jobhistory.recovery.store.fs.uri":"${hadoop.tmp.dir}/mapred/history/recoverystore","hadoop.proxyuser.hive.groups":"*","ha.zookeeper.session-timeout.ms":"5000","mapreduce.map.java.opts":"-Djava.net.preferIPv4Stack=true","tfile.io.chunk.size":"1048576","fs.s3a.s3guard.ddb.table.capacity.write":"100","mapreduce.job.speculative.slowtaskthreshold":"1.0","io.serializations":"org.apache.hadoop.io.serializer.WritableSerialization,org.apache.hadoop.io.serializer.avro.AvroSpecificSerialization,org.apache.hadoop.io.serializer.avro.AvroReflectSerialization","hadoop.security.kms.client.failover.sleep.max.millis":"2000","hadoop.security.group.mapping.ldap.directory.search.timeout":"10000","dfs.ha.automatic-failover.enabled":"false","mapreduce.job.counters.groups.max":"50","dfs.namenode.decommission.interval":"30","fs.swift.impl":"org.apache.hadoop.fs.swift.snative.SwiftNativeFileSystem","yarn.nodemanager.local-cache.max-files-per-directory":"8192","dfs.datanode.handler.count":"10","dfs.namenode.xattrs.enabled":"true","dfs.namenode.safemode.threshold-pct":"0.999f","dfs.client.socket.send.buffer.size":"0","mapreduce.map.sort.spill.percent":"0.8","yarn.resourcemanager.webapp.delegation-token-auth-filter.enabled":"*********(redacted)","hadoop.security.group.mapping.ldap.posix.attr.uid.name":"uidNumber","dfs.datanode.sync.behind.writes":"false","dfs.namenode.stale.datanode.interval":"30000","mapreduce.ifile.readahead":"true","yarn.timeline-service.leveldb-timeline-store.ttl-interval-ms":"300000","dfs.datanode.transfer.socket.send.buffer.size":"0","hadoop.security.kms.client.encrypted.key.cache.num.refill.threads":"2","dfs.namenode.reencrypt.throttle.limit.handler.ratio":"1.0","yarn.resourcemanager.scheduler.class":"org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler","yarn.app.mapreduce.am.command-opts":"-Djava.net.preferIPv4Stack=true -Xmx825955249","dfs.journalnode.https-address":"0.0.0.0:8481","mapreduce.cluster.local.dir":"${hadoop.tmp.dir}/mapred/local","hadoop.proxyuser.hue.hosts":"*","io.mapfile.bloom.error.rate":"0.005","dfs.user.home.dir.prefix":"/user","hadoop.proxyuser.hue.groups":"*","ha.failover-controller.graceful-fence.rpc-timeout.ms":"5000","ftp.replication":"3","mapreduce.jobtracker.persist.jobstatus.dir":"/jobtracker/jobsInfo","hadoop.security.uid.cache.secs":"14400","mapreduce.job.maxtaskfailures.per.tracker":"3","fs.s3a.metadatastore.impl":"org.apache.hadoop.fs.s3a.s3guard.NullMetadataStore","io.skip.checksum.errors":"false","dfs.namenode.snapshot.capture.openfiles":"false","dfs.datanode.directoryscan.interval":"21600","yarn.app.mapreduce.client-am.ipc.max-retries-on-timeouts":"3","dfs.client.read.shortcircuit.streams.cache.expiry.ms":"300000","fs.s3a.connection.timeout":"200000","mapreduce.job.max.split.locations":"10","dfs.namenode.write.stale.datanode.ratio":"0.5f","hadoop.registry.zk.session.timeout.ms":"60000","mapreduce.shuffle.transfer.buffer.size":"131072","yarn.timeline-service.client.retry-interval-ms":"1000","mapreduce.jobtracker.taskcache.levels":"2","yarn.http.policy":"HTTP_ONLY","fs.s3a.socket.send.buffer":"8192","hadoop.http.authentication.token.validity":"*********(redacted)","mapreduce.shuffle.max.connections":"0","mapreduce.job.emit-timeline-data":"false","hadoop.kerberos.min.seconds.before.relogin":"60","mapreduce.jobhistory.move.thread-count":"3","dfs.domain.socket.path":"/var/run/hdfs-sockets/dn","yarn.resourcemanager.admin.client.thread-count":"1","mapreduce.jobtracker.persist.jobstatus.active":"true","dfs.namenode.reencrypt.sleep.interval":"1m","fs.s3a.buffer.dir":"${hadoop.tmp.dir}/s3a","hadoop.ssl.enabled.protocols":"TLSv1,SSLv2Hello,TLSv1.1,TLSv1.2","mapreduce.jobhistory.admin.address":"test-1.vpc.company.com:10033","mapreduce.shuffle.port":"13562","yarn.resourcemanager.max-log-aggregation-diagnostics-in-memory":"10","yarn.nodemanager.health-checker.interval-ms":"600000","mapreduce.tasktracker.report.address":"127.0.0.1:0","dfs.namenode.edit.log.autoroll.multiplier.threshold":"2.0","io.seqfile.lazydecompress":"true","ftp.blocksize":"67108864","dfs.namenode.backup.http-address":"0.0.0.0:50105","dfs.disk.balancer.max.disk.throughputInMBperSec":"10","mapreduce.jobtracker.instrumentation":"org.apache.hadoop.mapred.JobTrackerMetricsInst","yarn.client.max-cached-nodemanagers-proxies":"0","yarn.nodemanager.delete.debug-delay-sec":"0","mapreduce.jobtracker.http.address":"0.0.0.0:50030","yarn.nodemanager.pmem-check-enabled":"true","yarn.nodemanager.disk-health-checker.max-disk-utilization-per-disk-percentage":"90.0","mapreduce.app-submission.cross-platform":"false","yarn.resourcemanager.work-preserving-recovery.scheduling-wait-ms":"10000","hadoop.security.groups.cache.secs":"300","yarn.resourcemanager.zk-retry-interval-ms":"1000","yarn.scheduler.increment-allocation-mb":"512","nfs.mountd.port":"4242","mapreduce.shuffle.max.threads":"0","hadoop.security.authorization":"false","mapreduce.job.complete.cancel.delegation.tokens":"*********(redacted)","fs.s3a.paging.maximum":"5000","nfs.exports.allowed.hosts":"* rw","mapreduce.jobhistory.http.policy":"HTTP_ONLY","dfs.datanode.dns.interface":"default","mapreduce.reduce.java.opts":"-Djava.net.preferIPv4Stack=true","s3native.replication":"3","hadoop.security.group.mapping.ldap.ssl":"false","dfs.namenode.fs-limits.max-xattrs-per-inode":"32","yarn.client.application-client-protocol.poll-interval-ms":"200","hadoop.proxyuser.flume.groups":"*","dfs.namenode.fs-limits.max-xattr-size":"16384","dfs.namenode.maintenance.replication.min":"1","dfs.client.write.exclude.nodes.cache.expiry.interval.millis":"600000","ha.zookeeper.parent-znode":"/hadoop-ha","dfs.namenode.safemode.extension":"30000","mapreduce.reduce.shuffle.merge.percent":"0.66","hadoop.security.group.mapping.ldap.search.filter.group":"(objectClass=group)","dfs.blocksize":"134217728","dfs.namenode.servicerpc-address":"test-1.vpc.company.com:8022","yarn.nodemanager.resourcemanager.minimum.version":"NONE","mapreduce.job.speculative.speculative-cap-running-tasks":"0.1","yarn.admin.acl":"*","yarn.resourcemanager.ha.automatic-failover.enabled":"true","mapreduce.reduce.skip.maxgroups":"0","mapreduce.reduce.shuffle.connect.timeout":"180000","yarn.resourcemanager.address":"test-1.vpc.company.com:8032","ipc.client.ping":"true","fs.adl.oauth2.access.token.provider.type":"*********(redacted)","dfs.namenode.resource.checked.volumes.minimum":"1","hadoop.proxyuser.HTTP.hosts":"*","mapreduce.shuffle.ssl.file.buffer.size":"65536","yarn.resourcemanager.ha.automatic-failover.embedded":"true","dfs.namenode.quota.init-threads":"4","dfs.journalnode.http-address":"0.0.0.0:8480","dfs.block.scanner.volume.bytes.per.second":"1048576","hadoop.ssl.enabled":"false","fs.s3a.multipart.purge":"false","dfs.storage.policy.enabled":"true","mapreduce.job.end-notification.max.attempts":"5","mapreduce.output.fileoutputformat.compress.codec":"org.apache.hadoop.io.compress.DefaultCodec","yarn.nodemanager.container-monitor.procfs-tree.smaps-based-rss.enabled":"false","dfs.namenode.edits.dir":"${dfs.namenode.name.dir}","ha.health-monitor.connect-retry-interval.ms":"1000","yarn.nodemanager.keytab":"/etc/krb5.keytab","dfs.namenode.support.allow.format":"true","dfs.ha.tail-edits.rolledits.timeout":"60","mapreduce.jobhistory.keytab":"/etc/security/keytab/jhs.service.keytab","fs.s3a.threads.max":"10","mapreduce.reduce.shuffle.input.buffer.percent":"0.70","mapreduce.cluster.temp.dir":"${hadoop.tmp.dir}/mapred/temp","s3.replication":"3","dfs.client.failover.connection.retries":"0","hadoop.tmp.dir":"/tmp/hadoop-${user.name}","mapreduce.job.maps":"2","dfs.namenode.secondary.http-address":"0.0.0.0:50090","mapreduce.job.end-notification.max.retry.interval":"5000","yarn.log-aggregation.retain-check-interval-seconds":"-1","yarn.resourcemanager.resource-tracker.client.thread-count":"50","nfs.wtmax":"1048576","yarn.timeline-service.leveldb-timeline-store.start-time-read-cache-size":"10000","nfs.dump.dir":"/tmp/.hdfs-nfs","yarn.resourcemanager.ha.automatic-failover.zk-base-path":"/yarn-leader-election","io.seqfile.local.dir":"${hadoop.tmp.dir}/io/local","mapreduce.client.submit.file.replication":"3","mapreduce.jobhistory.minicluster.fixed.ports":"false","fs.s3a.multipart.threshold":"128M","dfs.namenode.service.handler.count":"10","dfs.datanode.data.dir.perm":"700","mapreduce.jobhistory.done-dir":"${yarn.app.mapreduce.am.staging-dir}/history/done","dfs.namenode.name.dir":"file:///dataroot/dataroot/dfs/nn","yarn.resourcemanager.zk-acl":"world:anyone:rwcda","ipc.client.idlethreshold":"4000","yarn.nodemanager.linux-container-executor.cgroups.strict-resource-usage":"false","mapreduce.reduce.input.buffer.percent":"0.0","fs.ftp.host.port":"21","ipc.ping.interval":"60000","dfs.namenode.num.checkpoints.retained":"2","dfs.namenode.kerberos.internal.spnego.principal":"${dfs.web.authentication.kerberos.principal}","yarn.resourcemanager.admin.address":"test-1.vpc.company.com:8033","file.client-write-packet-size":"65536","hadoop.treat.subject.external":"false","ipc.client.kill.max":"10","mapreduce.reduce.speculative":"false","dfs.disk.balancer.plan.threshold.percent":"10","mapreduce.local.clientfactory.class.name":"org.apache.hadoop.mapred.LocalClientFactory","dfs.client.use.legacy.blockreader":"false","mapreduce.job.reducer.unconditional-preempt.delay.sec":"300","yarn.nodemanager.disk-health-checker.interval-ms":"120000","ipc.client.connection.maxidletime":"10000","mapreduce.task.io.sort.mb":"256","yarn.nodemanager.localizer.client.thread-count":"5","dfs.namenode.checkpoint.max-retries":"3","dfs.namenode.reject-unresolved-dn-topology-mapping":"false","dfs.namenode.delegation.token.max-lifetime":"*********(redacted)","dfs.client.block.write.replace-datanode-on-failure.min-replication":"0","yarn.nodemanager.localizer.cache.cleanup.interval-ms":"600000","hadoop.security.crypto.codec.classes.aes.ctr.nopadding":"org.apache.hadoop.crypto.OpensslAesCtrCryptoCodec,org.apache.hadoop.crypto.JceAesCtrCryptoCodec","fs.s3a.connection.ssl.enabled":"true","yarn.nodemanager.process-kill-wait.ms":"2000","dfs.namenode.num.extra.edits.retained":"1000000","mapreduce.job.hdfs-servers":"${fs.defaultFS}","yarn.scheduler.increment-allocation-vcores":"1","fs.df.interval":"60000","fs.s3.sleepTimeSeconds":"10","fs.s3a.multiobjectdelete.enable":"true","yarn.nodemanager.disk-health-checker.min-healthy-disks":"0.25","hadoop.shell.missing.defaultFs.warning":"true","io.file.buffer.size":"65536","hadoop.work.around.non.threadsafe.getpwuid":"false","dfs.permissions.superusergroup":"supergroup","hadoop.security.group.mapping.ldap.search.attr.member":"member","hadoop.security.random.device.file.path":"/dev/urandom","mapreduce.tasktracker.dns.interface":"default","hadoop.security.sensitive-config-keys":"*********(redacted)","fs.s3a.s3guard.ddb.max.retries":"9","hadoop.rpc.socket.factory.class.default":"org.apache.hadoop.net.StandardSocketFactory","dfs.permissions.enabled":"true","yarn.resourcemanager.connect.retry-interval.ms":"30000","yarn.scheduler.minimum-allocation-mb":"1024","yarn.app.mapreduce.am.staging-dir":"/user","mapreduce.reduce.shuffle.read.timeout":"180000","yarn.app.mapreduce.am.admin.user.env":"LD_LIBRARY_PATH=$HADOOP_COMMON_HOME/lib/native:$JAVA_LIBRARY_PATH","dfs.datanode.https.address":"0.0.0.0:50475","dfs.namenode.hosts.provider.classname":"org.apache.hadoop.hdfs.server.blockmanagement.HostFileManager","dfs.datanode.transfer.socket.recv.buffer.size":"0","fs.s3a.connection.establish.timeout":"5000","dfs.namenode.fslock.fair":"true","mapreduce.job.running.map.limit":"0","hadoop.ssl.require.client.cert":"false","hadoop.kerberos.kinit.command":"kinit","hadoop.fuse.connection.timeout":"300","mapreduce.reduce.log.level":"INFO","hadoop.security.dns.log-slow-lookups.threshold.ms":"1000","mapreduce.job.ubertask.enable":"false","adl.http.timeout":"5000","yarn.nodemanager.vmem-pmem-ratio":"2.1","dfs.client.slow.io.warning.threshold.ms":"30000","hadoop.rpc.protection":"authentication","ha.health-monitor.rpc-timeout.ms":"45000","s3native.stream-buffer-size":"4096","yarn.nodemanager.remote-app-log-dir":"/tmp/logs","fs.s3a.s3guard.cli.prune.age":"86400000","dfs.client.read.shortcircuit.streams.cache.size":"256","dfs.client.use.legacy.blockreader.local":"false","yarn.app.mapreduce.am.containerlauncher.threadpool-initial-size":"10","fs.s3n.multipart.uploads.enabled":"false","dfs.namenode.path.based.cache.retry.interval.ms":"30000","hadoop.security.crypto.buffer.size":"8192","yarn.client.failover-retries-on-socket-timeouts":"0","dfs.balancer.keytab.enabled":"false","hadoop.security.instrumentation.requires.admin":"false","yarn.nodemanager.delete.thread-count":"4","dfs.datanode.balance.bandwidthPerSec":"10485760","dfs.namenode.name.dir.restore":"false","hadoop.registry.jaas.context":"Client","dfs.client.failover.sleep.max.millis":"15000","yarn.timeline-service.leveldb-timeline-store.path":"${hadoop.tmp.dir}/yarn/timeline","s3.blocksize":"67108864","yarn.am.blacklisting.disable-failure-threshold":"0.8f","io.map.index.interval":"128","mapreduce.job.counters.max":"120","dfs.namenode.max-lock-hold-to-release-lease-ms":"25","dfs.namenode.datanode.registration.ip-hostname-check":"true","yarn.timeline-service.store-class":"org.apache.hadoop.yarn.server.timeline.LeveldbTimelineStore","mapreduce.jobhistory.move.interval-ms":"180000","dfs.namenode.resource.du.reserved":"104857600","dfs.datanode.bp-ready.timeout":"20","yarn.nodemanager.localizer.fetch.thread-count":"4","yarn.resourcemanager.scheduler.client.thread-count":"50","hadoop.ssl.hostname.verifier":"DEFAULT","dfs.namenode.full.block.report.lease.length.ms":"300000","mapreduce.tasktracker.instrumentation":"org.apache.hadoop.mapred.TaskTrackerMetricsInst","mapreduce.job.classloader":"false","mapreduce.task.profile.map.params":"${mapreduce.task.profile.params}","ipc.client.connect.timeout":"20000","s3.stream-buffer-size":"4096","yarn.resourcemanager.nm.liveness-monitor.interval-ms":"1000","yarn.nm.liveness-monitor.expiry-interval-ms":"600000","dfs.namenode.secondary.https-address":"0.0.0.0:50091","s3native.bytes-per-checksum":"512","dfs.namenode.fs-limits.max-directory-items":"1048576","nfs.server.port":"2049","dfs.namenode.delegation.token.renew-interval":"*********(redacted)","mapreduce.jobtracker.address":"local","yarn.nodemanager.recovery.enabled":"false","mapreduce.job.end-notification.retry.interval":"1000","fs.du.interval":"600000","dfs.namenode.list.openfiles.num.responses":"1000","hadoop.security.groups.cache.warn.after.ms":"5000","file.bytes-per-checksum":"512","dfs.namenode.blocks.per.postponedblocks.rescan":"10000","dfs.namenode.checkpoint.period":"3600","hadoop.security.groups.cache.background.reload":"false","yarn.resourcemanager.amlauncher.log.command":"false","net.topology.script.number.args":"100","mapreduce.task.merge.progress.records":"10000","yarn.nodemanager.localizer.address":"${yarn.nodemanager.hostname}:8040","yarn.timeline-service.keytab":"/etc/krb5.keytab","mapreduce.reduce.shuffle.fetch.retry.timeout-ms":"30000","dfs.namenode.snapshot.skip.capture.accesstime-only-change":"false","dfs.webhdfs.user.provider.user.pattern":"^[A-Za-z_][A-Za-z0-9._-]*[$]?$","dfs.webhdfs.acl.provider.permission.pattern":"^(default:)?(user|group|mask|other):[[A-Za-z_][A-Za-z0-9._-]]*:([rwx-]{3})?(,(default:)?(user|group|mask|other):[[A-Za-z_][A-Za-z0-9._-]]*:([rwx-]{3})?)*$","mapreduce.fileoutputcommitter.algorithm.version":"1","yarn.resourcemanager.work-preserving-recovery.enabled":"false","mapreduce.map.skip.maxrecords":"0","mapreduce.jobtracker.handler.count":"10","hadoop.http.authentication.type":"simple","mapreduce.job.jvm.numtasks":"1","hadoop.proxyuser.flume.hosts":"*","mapreduce.task.userlog.limit.kb":"0","yarn.resourcemanager.scheduler.monitor.enable":"false","fs.s3n.block.size":"67108864","ipc.client.connect.max.retries":"10","hadoop.registry.zk.retry.times":"5","mapreduce.jobtracker.staging.root.dir":"${hadoop.tmp.dir}/mapred/staging","dfs.namenode.http-address":"test-1.vpc.company.com:20101","mapreduce.jobtracker.jobhistory.lru.cache.size":"5","dfs.datanode.directoryscan.threads":"1","dfs.datanode.fsdatasetcache.max.threads.per.volume":"4","dfs.namenode.fs-limits.max-blocks-per-file":"1048576","dfs.disk.balancer.enabled":"false","mapreduce.shuffle.listen.queue.size":"128","mapreduce.tasktracker.local.dir.minspacestart":"0","mapreduce.map.cpu.vcores":"1","hadoop.user.group.static.mapping.overrides":"dr.who=;","dfs.datanode.cache.revocation.timeout.ms":"900000","mapreduce.jobhistory.recovery.store.class":"org.apache.hadoop.mapreduce.v2.hs.HistoryServerFileSystemStateStoreService","dfs.client.mmap.cache.size":"256","dfs.ha.log-roll.period":"120","dfs.client.failover.sleep.base.millis":"500","yarn.resourcemanager.fail-fast":"${yarn.fail-fast}","yarn.resourcemanager.proxy-user-privileges.enabled":"false","mapreduce.job.reducer.preempt.delay.sec":"0","hadoop.util.hash.type":"murmur","dfs.namenode.accesstime.precision":"3600000","yarn.app.mapreduce.client.job.max-retries":"3","mapreduce.reduce.shuffle.retry-delay.max.ms":"60000","mapreduce.task.profile.params":"-agentlib:hprof=cpu=samples,heap=sites,force=n,thread=y,verbose=n,file=%s","yarn.app.mapreduce.shuffle.log.backups":"0","hadoop.registry.zk.retry.interval.ms":"1000","fs.AbstractFileSystem.file.impl":"org.apache.hadoop.fs.local.LocalFs","yarn.nodemanager.log-aggregation.roll-monitoring-interval-seconds":"-1","dfs.client.context":"default","mapreduce.jobhistory.cleaner.interval-ms":"86400000","hadoop.registry.zk.quorum":"localhost:2181","mapreduce.output.fileoutputformat.compress":"false","yarn.resourcemanager.am-rm-tokens.master-key-rolling-interval-secs":"*********(redacted)","hadoop.ssl.server.conf":"ssl-server.xml","dfs.http.policy":"HTTP_ONLY","dfs.client.https.keystore.resource":"ssl-client.xml","mapreduce.client.completion.pollinterval":"5000","hadoop.ssl.keystores.factory.class":"org.apache.hadoop.security.ssl.FileBasedKeyStoresFactory","yarn.app.mapreduce.am.resource.cpu-vcores":"1","yarn.timeline-service.enabled":"false","yarn.acl.enable":"true","dfs.domain.socket.disable.interval.seconds":"1","dfs.image.transfer.chunksize":"65536","dfs.balancer.max-no-move-interval":"60000","mapreduce.tasktracker.map.tasks.maximum":"2","dfs.namenode.edits.journal-plugin.qjournal":"org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager","mapreduce.task.profile":"false","dfs.webhdfs.enabled":"true","yarn.resourcemanager.fs.state-store.uri":"${hadoop.tmp.dir}/yarn/system/rmstore","yarn.nodemanager.linux-container-executor.nonsecure-mode.local-user":"nobody","dfs.namenode.list.encryption.zones.num.responses":"100","yarn.resourcemanager.configuration.provider-class":"org.apache.hadoop.yarn.LocalConfigurationProvider","dfs.namenode.top.num.users":"10","dfs.disk.balancer.block.tolerance.percent":"10","yarn.nodemanager.resource.percentage-physical-cpu-limit":"100","mapreduce.jobhistory.client.thread-count":"10","tfile.fs.input.buffer.size":"262144","mapreduce.client.progressmonitor.pollinterval":"1000","yarn.nodemanager.log-dirs":"${yarn.log.dir}/userlogs","io.seqfile.sorter.recordlimit":"1000000","hadoop.security.auth_to_local":"DEFAULT","dfs.blockreport.initialDelay":"0","fs.automatic.close":"true","dfs.client.block.write.replace-datanode-on-failure.best-effort":"false","dfs.namenode.replication.min":"1","dfs.balancer.address":"0.0.0.0:0","fs.s3n.multipart.copy.block.size":"5368709120","yarn.nodemanager.hostname":"0.0.0.0","nfs.rtmax":"1048576","yarn.resourcemanager.zk-timeout-ms":"10000","ftp.stream-buffer-size":"4096","yarn.fail-fast":"false","hadoop.security.group.mapping.ldap.search.filter.user":"(&(objectClass=user)(sAMAccountName={0}))","dfs.datanode.directoryscan.throttle.limit.ms.per.sec":"1000","yarn.nodemanager.container-localizer.log.level":"INFO","yarn.timeline-service.address":"${yarn.timeline-service.hostname}:10200","dfs.namenode.replication.work.multiplier.per.iteration":"2","mapreduce.job.ubertask.maxmaps":"9","fs.s3a.threads.keepalivetime":"60","dfs.namenode.reencrypt.throttle.limit.updater.ratio":"1.0","dfs.namenode.avoid.write.stale.datanode":"false","dfs.short.circuit.shared.memory.watcher.interrupt.check.ms":"60000","dfs.datanode.available-space-volume-choosing-policy.balanced-space-preference-fraction":"0.75f","mapreduce.task.files.preserve.failedtasks":"false","yarn.app.mapreduce.client.job.retry-interval":"2000","ha.failover-controller.graceful-fence.connection.retries":"1","dfs.client.mmap.enabled":"true","mapreduce.reduce.cpu.vcores":"1","hadoop.proxyuser.oozie.groups":"*","fs.client.resolve.remote.symlinks":"true","dfs.image.compression.codec":"org.apache.hadoop.io.compress.DefaultCodec","mapreduce.jobtracker.restart.recover":"false","dfs.namenode.decommission.blocks.per.interval":"500000","mapreduce.tasktracker.reduce.tasks.maximum":"2","yarn.nodemanager.local-dirs":"${hadoop.tmp.dir}/nm-local-dir","mapreduce.shuffle.connection-keep-alive.enable":"false","fs.s3a.path.style.access":"false","yarn.nodemanager.aux-services.mapreduce_shuffle.class":"org.apache.hadoop.mapred.ShuffleHandler","fs.adl.impl":"org.apache.hadoop.fs.adl.AdlFileSystem","yarn.resourcemanager.nodemanager.minimum.version":"NONE","net.topology.impl":"org.apache.hadoop.net.NetworkTopology","io.map.index.skip":"0","dfs.namenode.safemode.min.datanodes":"0","fs.ftp.data.connection.mode":"ACTIVE_LOCAL_DATA_CONNECTION_MODE","mapreduce.job.userlog.retain.hours":"24","yarn.scheduler.maximum-allocation-vcores":"4","yarn.nodemanager.log-aggregation.compression-type":"none","dfs.namenode.enable.retrycache":"true","yarn.ipc.rpc.class":"org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC","dfs.namenode.startup.delay.block.deletion.sec":"0","mapreduce.reduce.maxattempts":"4","hadoop.security.dns.log-slow-lookups.enabled":"false","mapreduce.job.committer.setup.cleanup.needed":"true","dfs.datanode.readahead.bytes":"4194304","mapreduce.jobtracker.heartbeats.in.second":"100","mapreduce.job.running.reduce.limit":"0","mapreduce.job.token.tracking.ids.enabled":"*********(redacted)","mapreduce.task.tmp.dir":"./tmp","hadoop.registry.system.acls":"sasl:yarn@, sasl:mapred@, sasl:mapred@hdfs@","yarn.nodemanager.recovery.dir":"${hadoop.tmp.dir}/yarn-nm-recovery","fs.s3a.fast.upload.buffer":"disk","mapreduce.jobhistory.intermediate-done-dir":"${yarn.app.mapreduce.am.staging-dir}/history/done_intermediate","yarn.app.mapreduce.shuffle.log.separate":"true","dfs.namenode.delegation.key.update-interval":"86400000","fs.s3a.max.total.tasks":"5","dfs.client.file-block-storage-locations.num-threads":"10","mapreduce.tasktracker.healthchecker.interval":"60000","fs.s3a.readahead.range":"64K","hadoop.http.authentication.simple.anonymous.allowed":"true","fs.s3a.fast.upload":"false","fs.s3a.attempts.maximum":"20","dfs.namenode.avoid.read.stale.datanode":"false","hadoop.registry.zk.connection.timeout.ms":"15000","dfs.https.port":"20102","yarn.nodemanager.health-checker.script.timeout-ms":"1200000","yarn.timeline-service.leveldb-timeline-store.start-time-write-cache-size":"10000","mapreduce.map.log.level":"INFO","mapreduce.output.fileoutputformat.compress.type":"BLOCK","hadoop.registry.rm.enabled":"false","mapreduce.ifile.readahead.bytes":"4194304","mapreduce.tasktracker.tasks.sleeptimebeforesigkill":"5000","yarn.resourcemanager.fs.state-store.retry-policy-spec":"2000, 500","dfs.namenode.posix.acl.inheritance.enabled":"false","dfs.blockreport.intervalMsec":"21600000","yarn.nodemanager.linux-container-executor.nonsecure-mode.limit-users":"true","mapreduce.cluster.acls.enabled":"false","mapreduce.job.speculative.retry-after-no-speculate":"1000","dfs.namenode.path.based.cache.refresh.interval.ms":"30000","dfs.namenode.edekcacheloader.interval.ms":"1000","file.stream-buffer-size":"4096","mapreduce.map.output.compress.codec":"org.apache.hadoop.io.compress.SnappyCodec","mapreduce.map.speculative":"false","dfs.disk.balancer.max.disk.errors":"5","dfs.datanode.use.datanode.hostname":"false","mapreduce.job.speculative.retry-after-speculate":"15000","hadoop.proxyuser.hdfs.hosts":"*","dfs.namenode.fs-limits.min-block-size":"1048576","yarn.nodemanager.linux-container-executor.cgroups.mount":"false","yarn.app.mapreduce.am.container.log.backups":"0","mapreduce.job.reduce.slowstart.completedmaps":"0.8","dfs.client.read.shortcircuit":"false","yarn.timeline-service.http-authentication.type":"simple","hadoop.security.group.mapping.ldap.search.attr.group.name":"cn","hadoop.proxyuser.yarn.groups":"*","dfs.client.cached.conn.retry":"3","dfs.namenode.invalidate.work.pct.per.iteration":"0.32f","hadoop.http.logs.enabled":"true","fs.s3a.block.size":"32M","yarn.nodemanager.logaggregation.threadpool-size-max":"100","dfs.replication.max":"512","dfs.namenode.inotify.max.events.per.rpc":"1000","yarn.resourcemanager.hostname":"0.0.0.0","mapreduce.reduce.shuffle.fetch.retry.enabled":"${yarn.nodemanager.recovery.enabled}","mapreduce.map.memory.mb":"0","mapreduce.task.skip.start.attempts":"2","fs.AbstractFileSystem.hdfs.impl":"org.apache.hadoop.fs.Hdfs","ipc.client.rpc-timeout.ms":"0","fs.s3.maxRetries":"4","dfs.default.chunk.view.size":"32768","mapreduce.input.lineinputformat.linespermap":"1","ipc.client.connect.max.retries.on.timeouts":"45","yarn.timeline-service.leveldb-timeline-store.read-cache-size":"104857600","fs.AbstractFileSystem.har.impl":"org.apache.hadoop.fs.HarFs","mapreduce.job.split.metainfo.maxsize":"10000000","yarn.am.liveness-monitor.expiry-interval-ms":"600000","dfs.client.mmap.retry.timeout.ms":"300000","yarn.resourcemanager.container-tokens.master-key-rolling-interval-secs":"*********(redacted)","dfs.namenode.list.cache.directives.num.responses":"100","fs.s3a.socket.recv.buffer":"8192","dfs.image.compress":"false","dfs.namenode.kerberos.principal.pattern":"*","yarn.application.classpath":"$HADOOP_CLIENT_CONF_DIR,$HADOOP_CONF_DIR,$HADOOP_COMMON_HOME/*,$HADOOP_COMMON_HOME/lib/*,$HADOOP_HDFS_HOME/*,$HADOOP_HDFS_HOME/lib/*,$HADOOP_YARN_HOME/*,$HADOOP_YARN_HOME/lib/*","fs.s3n.multipart.uploads.block.size":"67108864","mapreduce.tasktracker.http.address":"0.0.0.0:50060","yarn.resourcemanager.resource-tracker.address":"test-1.vpc.company.com:8031","hadoop.fuse.timer.period":"5","mapreduce.job.heap.memory-mb.ratio":"0.8","dfs.datanode.hdfs-blocks-metadata.enabled":"true","dfs.namenode.checkpoint.dir":"file://${hadoop.tmp.dir}/dfs/namesecondary","dfs.datanode.max.transfer.threads":"4096","dfs.namenode.edits.asynclogging":"true","nfs.allow.insecure.ports":"true","mapreduce.client.output.filter":"FAILED","hadoop.http.filter.initializers":"org.apache.hadoop.http.lib.StaticUserWebFilter","mapreduce.reduce.memory.mb":"0","s3native.client-write-packet-size":"65536","mapreduce.admin.user.env":"LD_LIBRARY_PATH=$HADOOP_COMMON_HOME/lib/native:$JAVA_LIBRARY_PATH","yarn.timeline-service.hostname":"0.0.0.0","file.replication":"1","yarn.nodemanager.container-metrics.unregister-delay-ms":"10000","hadoop.proxyuser.mapred.hosts":"*","hadoop.proxyuser.oozie.hosts":"*","yarn.nodemanager.log.retain-seconds":"10800","hadoop.proxyuser.mapred.groups":"*","yarn.resourcemanager.keytab":"/etc/krb5.keytab","mapreduce.reduce.merge.inmem.threshold":"1000","dfs.client.https.need-auth":"false","dfs.blockreport.split.threshold":"1000000","dfs.client.block.write.replace-datanode-on-failure.policy":"DEFAULT","mapreduce.shuffle.ssl.enabled":"false","dfs.namenode.write-lock-reporting-threshold-ms":"5000","dfs.block.access.token.enable":"*********(redacted)","yarn.resourcemanager.state-store.max-completed-applications":"${yarn.resourcemanager.max-completed-applications}","httpfs.buffer.size":"4096","dfs.client.file-block-storage-locations.timeout.millis":"1000","dfs.namenode.block-placement-policy.default.prefer-local-node":"true","mapreduce.job.speculative.minimum-allowed-tasks":"10","yarn.log-aggregation.retain-seconds":"-1","dfs.namenode.replication.considerLoad":"true","yarn.nodemanager.disk-health-checker.min-free-space-per-disk-mb":"0","mapreduce.jobhistory.max-age-ms":"604800000","hadoop.proxyuser.hdfs.groups":"*","dfs.namenode.retrycache.heap.percent":"0.03f","dfs.datanode.cache.revocation.polling.ms":"500","mapreduce.jobhistory.webapp.address":"test-1.vpc.company.com:19888","dfs.namenode.path.based.cache.block.map.allocation.percent":"0.25","mapreduce.jobtracker.system.dir":"${hadoop.tmp.dir}/mapred/system","mapreduce.tasktracker.taskmemorymanager.monitoringinterval":"5000","dfs.journalnode.rpc-address":"0.0.0.0:8485","yarn.client.nodemanager-connect.max-wait-ms":"180000","yarn.resourcemanager.webapp.address":"test-1.vpc.company.com:8088","mapreduce.jobhistory.recovery.enable":"false","dfs.client.short.circuit.replica.stale.threshold.ms":"1800000","mapreduce.reduce.shuffle.parallelcopies":"10","fs.trash.interval":"1","dfs.namenode.replication.interval":"3","yarn.app.mapreduce.client.max-retries":"3","hadoop.security.authentication":"simple","dfs.namenode.top.enabled":"true","mapreduce.task.profile.reduce.params":"${mapreduce.task.profile.params}","dfs.datanode.du.reserved":"0","yarn.app.mapreduce.am.resource.mb":"1024","mapreduce.input.fileinputformat.list-status.num-threads":"1","dfs.namenode.lazypersist.file.scrub.interval.sec":"300","yarn.nodemanager.container-executor.class":"org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor","io.mapfile.bloom.size":"1048576","yarn.timeline-service.ttl-ms":"604800000","yarn.nodemanager.resource.cpu-vcores":"8","mapreduce.job.reduces":"6","fs.s3a.multipart.size":"64M","yarn.scheduler.minimum-allocation-vcores":"1","dfs.namenode.reencrypt.batch.size":"1000","mapreduce.job.speculative.speculative-cap-total-tasks":"0.01","dfs.datanode.http.address":"0.0.0.0:50075","hadoop.ssl.client.conf":"ssl-client.xml","mapreduce.job.queuename":"default","fs.s3a.metadatastore.authoritative":"false","ha.health-monitor.sleep-after-disconnect.ms":"1000","s3.bytes-per-checksum":"512","yarn.app.mapreduce.shuffle.log.limit.kb":"0","dfs.namenode.list.cache.pools.num.responses":"100","hadoop.security.group.mapping":"org.apache.hadoop.security.ShellBasedUnixGroupsMapping","mapreduce.jobhistory.jhist.format":"binary","yarn.resourcemanager.ha.enabled":"false","dfs.encrypt.data.transfer":"false","hadoop.http.staticuser.user":"dr.who","mapreduce.task.exit.timeout.check-interval-ms":"20000","mapreduce.task.exit.timeout":"60000","yarn.nodemanager.linux-container-executor.resources-handler.class":"org.apache.hadoop.yarn.server.nodemanager.util.DefaultLCEResourcesHandler","mapreduce.reduce.shuffle.memory.limit.percent":"0.25","mapreduce.job.redacted-properties":"*********(redacted)","dfs.namenode.top.windows.minutes":"1,5,25","s3.client-write-packet-size":"65536","mapreduce.map.output.compress":"true","ha.zookeeper.acl":"world:anyone:rwcda","ipc.server.max.connections":"0","yarn.scheduler.maximum-allocation-mb":"12288","yarn.resourcemanager.scheduler.monitor.policies":"org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy","yarn.app.mapreduce.am.container.log.limit.kb":"0","s3native.blocksize":"67108864","ipc.client.connect.retry.interval":"1000","hadoop.proxyuser.httpfs.groups":"*","yarn.resourcemanager.zk-state-store.parent-path":"/rmstore","dfs.namenode.edit.log.autoroll.check.interval.ms":"300000","mapreduce.jobhistory.cleaner.enable":"true","hadoop.security.kms.client.encrypted.key.cache.expiry":"43200000","hadoop.proxyuser.httpfs.hosts":"*","dfs.client.use.datanode.hostname":"false","dfs.stream-buffer-size":"4096","yarn.client.nodemanager-client-async.thread-pool-max-size":"500","mapreduce.map.maxattempts":"4","dfs.datanode.drop.cache.behind.writes":"false","mapreduce.tasktracker.dns.nameserver":"default","yarn.nodemanager.sleep-delay-before-sigkill.ms":"250","mapreduce.job.end-notification.retry.attempts":"0","hadoop.proxyuser.yarn.hosts":"*","yarn.resourcemanager.zk-num-retries":"1000","dfs.client.failover.max.attempts":"15","mapreduce.tasktracker.indexcache.mb":"10","hadoop.registry.zk.root":"/registry","adl.feature.ownerandgroup.enableupn":"false","mapreduce.job.reduce.shuffle.consumer.plugin.class":"org.apache.hadoop.mapreduce.task.reduce.Shuffle","yarn.resourcemanager.delayed.delegation-token.removal-interval-ms":"*********(redacted)","dfs.namenode.snapshotdiff.allow.snap-root-descendant":"true","yarn.nodemanager.localizer.cache.target-size-mb":"10240","zlib.compress.level":"DEFAULT_COMPRESSION","ftp.client-write-packet-size":"65536","mapreduce.jobtracker.maxtasks.perjob":"-1","fs.AbstractFileSystem.adl.impl":"org.apache.hadoop.fs.adl.Adl","hadoop.proxyuser.hive.hosts":"*","dfs.block.access.token.lifetime":"*********(redacted)","dfs.namenode.max.extra.edits.segments.retained":"10000","yarn.client.failover-retries":"0","fs.s3a.multipart.purge.age":"86400","dfs.image.transfer.bandwidthPerSec":"0","io.native.lib.available":"true","net.topology.node.switch.mapping.impl":"org.apache.hadoop.net.ScriptBasedMapping","ipc.server.listen.queue.size":"128","dfs.namenode.edekcacheloader.initial.delay.ms":"3000","map.sort.class":"org.apache.hadoop.util.QuickSort","dfs.namenode.acls.enabled":"false","hadoop.security.kms.client.authentication.retry-count":"1","fs.permissions.umask-mode":"022","dfs.datanode.ipc.address":"0.0.0.0:50020","yarn.nodemanager.vmem-check-enabled":"false","yarn.nodemanager.recovery.compaction-interval-secs":"3600","yarn.app.mapreduce.client-am.ipc.max-retries":"3","dfs.lock.suppress.warning.interval":"10s","dfs.client.block.write.retries":"3","mapreduce.job.ubertask.maxreduces":"1","hadoop.security.kms.client.encrypted.key.cache.size":"500","hadoop.security.java.secure.random.algorithm":"SHA1PRNG","ha.failover-controller.cli-check.rpc-timeout.ms":"20000","mapreduce.application.classpath":"$HADOOP_MAPRED_HOME/*,$HADOOP_MAPRED_HOME/lib/*,$MR2_CLASSPATH","yarn.client.nodemanager-connect.retry-interval-ms":"10000","dfs.client-write-packet-size":"65536","yarn.nodemanager.env-whitelist":"JAVA_HOME,HADOOP_COMMON_HOME,HADOOP_HDFS_HOME,HADOOP_CONF_DIR,HADOOP_YARN_HOME","dfs.datanode.dns.nameserver":"default","yarn.nodemanager.webapp.address":"${yarn.nodemanager.hostname}:8042","rpc.metrics.quantile.enable":"false","mapreduce.jobhistory.admin.acl":"*","yarn.resourcemanager.system-metrics-publisher.dispatcher.pool-size":"10","hadoop.http.authentication.kerberos.keytab":"${user.home}/hadoop.keytab","dfs.image.transfer.timeout":"60000","yarn.resourcemanager.recovery.enabled":"false","dfs.datanode.available-space-volume-choosing-policy.balanced-space-threshold":"10737418240","dfs.client.failover.connection.retries.on.timeouts":"0"},"System Properties":{"java.io.tmpdir":"/tmp","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.security.egd":"file:///dev/urandom","java.vm.specification.version":"1.8","user.home":"/home/systest","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/usr/java/jdk1.8.0_144/jre/lib/amd64","user.dir":"/tmp","java.library.path":":/opt/cloudera/parcels/CDH/lib/hadoop/lib/native:/opt/cloudera/parcels/CDH/lib/hadoop/lib/native:/opt/cloudera/parcels/CDH/lib/hadoop/lib/native:/usr/java/packages/lib/amd64:/usr/lib64:/lib64:/lib:/usr/lib","sun.cpu.isalist":"","os.arch":"amd64","java.vm.version":"25.144-b01","jetty.git.hash":"27208684755d94a92186989f695db2d7b21ebc51","java.endorsed.dirs":"/usr/java/jdk1.8.0_144/jre/lib/endorsed","java.runtime.version":"1.8.0_144-b01","java.vm.info":"mixed mode","java.ext.dirs":"/usr/java/jdk1.8.0_144/jre/lib/ext:/usr/java/packages/lib/ext","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"52.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/usr/java/jdk1.8.0_144/jre/lib/resources.jar:/usr/java/jdk1.8.0_144/jre/lib/rt.jar:/usr/java/jdk1.8.0_144/jre/lib/sunrsasign.jar:/usr/java/jdk1.8.0_144/jre/lib/jsse.jar:/usr/java/jdk1.8.0_144/jre/lib/jce.jar:/usr/java/jdk1.8.0_144/jre/lib/charsets.jar:/usr/java/jdk1.8.0_144/jre/lib/jfr.jar:/usr/java/jdk1.8.0_144/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Los_Angeles","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"3.10.0-514.26.2.el7.x86_64","sun.os.patch.level":"unknown","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","user.language":"en","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.print.PSPrinterJob","java.awt.graphicsenv":"sun.awt.X11GraphicsEnvironment","awt.toolkit":"sun.awt.X11.XToolkit","os.name":"Linux","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"systest","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master yarn --deploy-mode client --conf spark.driver.memory=2g --conf spark.executor.heartbeatInterval=1000 --conf spark.executor.metrics.pollingInterval=100 --conf spark.eventLog.logStageExecutorProcessTreeMetrics.enabled=true --conf spark.yarn.maxAppAttempts=1 --conf spark.locality.wait.process=0 --conf spark.executor.memoryOverhead=1024 --conf spark.executor.extraJavaOptions=-Djava.security.egd=file:///dev/urandom --conf spark.eventLog.logStageExecutorMetrics.enabled=true --conf spark.driver.extraJavaOptions=-Djava.security.egd=file:///dev/urandom --class com.company.spark.LargeBlocks --num-executors 3 --executor-memory 7g /tmp/__spark_test__/spark3-tests-0.1.0-cdh5.9.0-SNAPSHOT-jar-with-dependencies.jar --targetBlockSizeGb 2.5 --taskSleepMillis 200 --doCache true --cacheOnDisk true --replicas 1 --concurrentReadJobs 2","java.home":"/usr/java/jdk1.8.0_144/jre","java.version":"1.8.0_144","sun.io.unicode.encoding":"UnicodeLittle"},"Classpath Entries":{"/opt/cloudera/parcels/CDH/jars/jackson-mapper-asl-1.8.8.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/joni-2.1.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jackson-core-2.2.3.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-external-blockcache-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/metrics-json-3.1.5.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/xz-1.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jackson-annotations-2.2.3.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/cglib-2.2.1-v20090111.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-procedure-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/parquet-jackson-1.10.1.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/asm-3.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/xml-apis-1.3.04.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/metrics-jvm-3.1.5.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-dbcp-1.4.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-streaming_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-xml-9.4.12.v20180830.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/minlog-1.3.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hsqldb-1.8.0.10.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/datanucleus-rdbms-3.2.9.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/pmml-model-1.4.8.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spire-macros_2.12-0.13.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/zstd-jni-1.3.2-2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/httpcore-4.2.5.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jta-1.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-yarn_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-logging-1.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/activation-1.1.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/xbean-asm7-shaded-4.12.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/kafka-0.9/paranamer-2.8.jar":"System Classpath","/opt/cloudera/parcels/CDH/lib/hadoop/NOTICE.txt":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/arrow-format-0.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/httpclient-4.2.5.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/netty-3.9.9.Final.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/htrace-core-3.1.0-incubating.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/javax.ws.rs-api-2.0.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jersey-container-servlet-core-2.22.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-mapreduce-client-nativetask-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jersey-client-2.22.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jackson-xc-1.8.8.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-configuration-1.6.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/commons-math3-3.1.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jsp-api-2.1.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-auth-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-yarn-registry-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/JavaEWAH-0.3.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/metrics-graphite-3.1.5.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/joda-time-2.9.9.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/aopalliance-1.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-gridmix-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/commons-compress-1.4.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/javolution-5.5.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-beanutils-1.7.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/flatbuffers-java-1.9.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/core-1.1.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/antlr-runtime-3.4.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-hdfs-nfs-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-hdfs-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-net-3.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/curator-framework-2.7.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/snappy-java-1.1.7.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-assembly_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-examples-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/shapeless_2.12-2.3.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-mapreduce-client-hs-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/xercesImpl-2.9.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jersey-server-2.22.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-lang-2.6.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jtransforms-2.4.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/snappy-0.2.jar":"System Classpath","/etc/spark2/conf/yarn-conf/":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-cli-1.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jackson-core-2.9.8.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/javax.annotation-api-1.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-util-6.1.26.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/stax-api-1.0-2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/javassist-3.18.1-GA.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-yarn-server-applicationhistoryservice-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/kafka-0.9/kafka-clients-0.9.0-kafka-2.0.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/guice-3.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/antlr4-runtime-4.7.1.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/core-3.1.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/opencsv-2.3.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-datajoin-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jersey-common-2.22.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jdo-api-3.0.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-webapp-9.4.12.v20180830.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-compiler-3.0.11.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-annotations-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/libthrift-0.9.3.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/kafka-0.9/kafka_2.11-0.9.0-kafka-2.0.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-azure-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-proxy-9.4.12.v20180830.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/antlr-2.7.7.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jettison-1.1.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jackson-core-asl-1.8.8.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/libfb303-0.9.3.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/zookeeper-3.4.5-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-resource-bundle-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jaxb-api-2.2.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-streaming-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-hadoop2-compat-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/pyrolite-4.13.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/activation-1.1.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-mapreduce-client-common-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/aircompressor-0.10.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-ant-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jline-2.11.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/netty-all-4.0.23.Final.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jersey-media-jaxb-2.22.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-servlets-9.4.12.v20180830.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/scala-xml_2.12-1.0.6.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-graphx_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-mapreduce-client-hs-plugins-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-yarn-server-web-proxy-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/commons-math-2.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/avro-1.8.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-kvstore_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/objenesis-2.5.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/stream-2.7.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-aws-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jackson-module-scala_2.12-2.9.8.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/spymemcached-2.11.6.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/api-util-1.0.0-M20.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/parquet-hadoop-1.10.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/orc-core-1.5.5-nohive.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/mockito-all-1.8.5.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jets3t-0.9.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jackson-xc-1.9.13.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-rsgroup-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/hive-metastore-1.2.1.spark2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/py4j-0.10.8.1.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-yarn-applications-distributedshell-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/osgi-resource-locator-1.0.1.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-rumen-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jsch-0.1.42.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/kafka-0.9/zkclient-0.7.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/xmlenc-0.52.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-io-2.4.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/metrics-core-3.1.5.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jamon-runtime-2.4.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/javax.inject-2.4.0-b34.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-common-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/parquet-common-1.10.1.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/netty-3.10.5.Final.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-mapreduce-client-app-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/machinist_2.12-0.6.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-util-9.4.12.v20180830.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-network-shuffle_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/chill_2.12-0.9.3.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/findbugs-annotations-1.3.9-1.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-rest-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-hadoop-compat-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/ST4-4.0.4.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jruby-cloudera-1.0.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-hive_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/leveldbjni-all-1.8.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-core_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-distcp-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-server-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/kafka-0.9/metrics-core-2.2.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/htrace-core-3.2.0-incubating.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/orc-mapreduce-1.5.5-nohive.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jcl-over-slf4j-1.7.16.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-digester-1.8.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-http-9.4.12.v20180830.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/json4s-jackson_2.12-3.5.3.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/metrics-core-2.2.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-math3-3.4.1.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/slf4j-log4j12-1.7.5.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-mapreduce-client-jobclient-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-shell-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-server-9.4.12.v20180830.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/chill-java-0.9.3.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/hk2-locator-2.4.0-b34.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/stax-api-1.0.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/scala-parser-combinators_2.12-1.1.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/RoaringBitmap-0.5.11.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jsp-api-2.1-6.1.14.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/log4j-1.2.16.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/javax.inject-1.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jasper-compiler-5.5.23.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-security-9.4.12.v20180830.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-mapreduce-client-shuffle-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-nfs-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/netty-all-4.1.30.Final.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jaxb-api-2.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/breeze-macros_2.12-0.13.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jodd-core-3.5.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/azure-data-lake-store-sdk-2.2.9.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/guava-12.0.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-codec-1.10.jar":"System Classpath","/usr/java/jdk1.8.0_144/lib/tools.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-yarn-server-nodemanager-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-sql_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/hive-exec-1.2.1.spark2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jcodings-1.0.8.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-azure-datalake-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/guava-11.0.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/avro-mapred-1.8.2-hadoop2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-catalyst_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-io-9.4.12.v20180830.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-yarn-applications-unmanaged-am-launcher-2.6.0-cdh5.15.2.jar":"System Classpath","/etc/spark2/conf/":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-sslengine-6.1.26.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/httpclient-4.5.6.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-continuation-9.4.12.v20180830.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/json4s-ast_2.12-3.5.3.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-jndi-9.4.12.v20180830.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jsr305-3.0.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/avro-1.7.6-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/univocity-parsers-2.7.3.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-annotations-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/api-asn1-api-1.0.0-M20.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jettison-1.3.3.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/hk2-utils-2.4.0-b34.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/hppc-0.7.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/logredactor-1.0.3.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/arrow-vector-0.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jaxb-impl-2.2.3-1.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-archives-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-plus-9.4.12.v20180830.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-client-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hue-plugins-3.9.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/high-scale-lib-1.1.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jackson-jaxrs-1.9.13.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-collections-3.2.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/scala-library-2.12.8.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/java-xmlbuilder-0.4.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/apache-log4j-extras-1.2.17.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/parquet-format-2.4.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/paranamer-2.8.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spire_2.12-0.13.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/disruptor-3.3.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-mllib-local_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-yarn-common-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-pool-1.5.4.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-servlet-9.4.12.v20180830.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/arpack_combined_all-0.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jackson-annotations-2.9.8.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jsp-2.1-6.1.14.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/log4j-1.2.17.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-repl_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/derby-10.12.1.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/orc-shims-1.5.5.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-mapreduce-client-core-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/aopalliance-repackaged-2.4.0-b34.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/hk2-api-2.4.0-b34.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/commons-daemon-1.0.13.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/ivy-2.4.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hamcrest-core-1.3.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-lang3-3.8.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/kafka-0.9/spark-streaming-kafka-0-8_2.11-2.4.0.cloudera1-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jackson-databind-2.2.3.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/kryo-shaded-4.0.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/scala-reflect-2.12.8.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-mapreduce-examples-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/httpcore-4.4.10.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/arrow-memory-0.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/lib/hadoop/LICENSE.txt":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-compress-1.8.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/curator-client-2.7.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/oro-2.0.8.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-thrift-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/avro-ipc-1.8.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-httpclient-3.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/parquet-encoding-1.10.1.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/okhttp-2.4.0.jar":"System Classpath","spark://test-1.vpc.company.com:34194/jars/spark3-tests-0.1.0-cdh5.9.0-SNAPSHOT-jar-with-dependencies.jar":"Added By User","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/scala-compiler-2.12.8.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-prefix-tree-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-6.1.26.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-yarn-api-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/commons-codec-1.9.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-beanutils-core-1.8.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/kafka-0.9/lz4-1.3.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/janino-3.0.11.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/libthrift-0.12.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-launcher_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/json4s-core_2.12-3.5.3.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/lz4-java-1.5.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jackson-jaxrs-1.8.8.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-network-common_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/macro-compat_2.12-1.1.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/breeze_2.12-0.13.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-unsafe_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-sls-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/snappy-java-1.0.4.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jersey-guava-2.22.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/validation-api-1.1.0.Final.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/microsoft-windowsazure-storage-sdk-0.6.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/compress-lzf-1.0.3.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-openstack-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-mllib_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-client-9.4.12.v20180830.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-it-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-extras-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jersey-container-servlet-2.22.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/javax.servlet-api-3.1.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/curator-recipes-2.7.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/parquet-column-1.10.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/unused-1.0.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-crypto-1.0.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jackson-databind-2.9.8.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/commons-codec-1.4.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/htrace-core4-4.0.1-incubating.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/guava-14.0.1.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/aws-java-sdk-bundle-1.11.134.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/zookeeper-3.4.6.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jackson-module-paranamer-2.9.8.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-protocol-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jasper-runtime-5.5.23.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/json4s-scalap_2.12-3.5.3.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/commons-logging-1.1.3.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-yarn-server-resourcemanager-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-archive-logs-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/paranamer-2.3.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/metrics-core-3.0.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/commons-beanutils-1.9.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jul-to-slf4j-1.7.16.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/stringtemplate-3.2.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/gson-2.2.4.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/okio-1.4.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/slf4j-api-1.7.5.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jsr305-3.0.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/bonecp-0.8.0.RELEASE.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-sketch_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/commons-el-1.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-tags_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/datanucleus-core-3.2.10.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/protobuf-java-2.5.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-yarn-server-common-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/parquet-hadoop-bundle-1.6.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-common-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/xz-1.5.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-yarn-client-2.6.0-cdh5.15.2.jar":"System Classpath"}} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/usr/java/jdk1.8.0_144/jre","Java Version":"1.8.0_144 (Oracle Corporation)","Scala Version":"version 2.12.8"},"Spark Properties":{"spark.lineage.log.dir":"/var/log/spark2/lineage","spark.serializer":"org.apache.spark.serializer.KryoSerializer","spark.yarn.jars":"local:/opt/cloudera/parcels/SPARK2/lib/spark2/jars/*","spark.executor.extraJavaOptions":"-Djava.security.egd=file:///dev/urandom","spark.driver.host":"test-1.vpc.company.com","spark.eventLog.enabled":"true","spark.executor.heartbeatInterval":"1000","spark.executor.memoryOverhead":"1024","spark.driver.port":"34194","spark.shuffle.service.enabled":"false","spark.driver.extraLibraryPath":"/opt/cloudera/parcels/CDH/lib/hadoop/lib/native","spark.lineage.enabled":"false","spark.jars":"file:/tmp/__spark_test__/spark3-tests-0.1.0-cdh5.9.0-SNAPSHOT-jar-with-dependencies.jar","spark.executor.metrics.pollingInterval":"100","spark.yarn.historyServer.address":"http://test-1.vpc.company.com:18089","spark.ui.enabled":"true","spark.app.name":"LargeBlocks","spark.ui.killEnabled":"true","spark.sql.hive.metastore.jars":"${env:HADOOP_COMMON_HOME}/../hive/lib/*:${env:HADOOP_COMMON_HOME}/client/*","spark.locality.wait.process":"0","spark.dynamicAllocation.schedulerBacklogTimeout":"1","spark.yarn.am.extraLibraryPath":"/opt/cloudera/parcels/CDH/lib/hadoop/lib/native","spark.scheduler.mode":"FIFO","spark.eventLog.logStageExecutorMetrics":"true","spark.driver.memory":"2g","spark.executor.instances":"3","spark.submit.pyFiles":"","spark.yarn.config.gatewayPath":"/opt/cloudera/parcels","spark.executor.id":"driver","spark.yarn.config.replacementPath":"{{HADOOP_COMMON_HOME}}/../../..","spark.driver.extraJavaOptions":"-Djava.security.egd=file:///dev/urandom","spark.eventLog.logStageExecutorProcessTreeMetrics.enabled":"true","spark.submit.deployMode":"client","spark.shuffle.service.port":"7337","spark.yarn.maxAppAttempts":"1","spark.master":"yarn","spark.authenticate":"false","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.network.crypto.enabled":"false","spark.executor.extraLibraryPath":"/opt/cloudera/parcels/CDH/lib/hadoop/lib/native","spark.executor.memory":"7g","spark.io.encryption.enabled":"false","spark.eventLog.dir":"hdfs://test-1.vpc.company.com:8020/user/spark/spark2ApplicationHistory","spark.dynamicAllocation.enabled":"false","spark.sql.catalogImplementation":"hive","spark.executor.cores":"1","spark.driver.appUIAddress":"http://test-1.vpc.company.com:4040","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"test-1.vpc.company.com","spark.dynamicAllocation.minExecutors":"0","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://test-1.vpc.company.com:8088/proxy/application_1553914137147_0018","spark.dynamicAllocation.executorIdleTimeout":"60","spark.app.id":"application_1553914137147_0018","spark.sql.hive.metastore.version":"1.1.0"},"Hadoop Properties":{"yarn.resourcemanager.amlauncher.thread-count":"50","dfs.namenode.resource.check.interval":"5000","fs.s3a.connection.maximum":"100","mapreduce.jobtracker.jobhistory.task.numberprogresssplits":"12","dfs.data.transfer.server.tcpnodelay":"true","mapreduce.tasktracker.healthchecker.script.timeout":"600000","fs.s3a.impl":"org.apache.hadoop.fs.s3a.S3AFileSystem","yarn.app.mapreduce.am.scheduler.heartbeat.interval-ms":"1000","hadoop.security.kms.client.timeout":"60","hadoop.http.authentication.kerberos.principal":"HTTP/_HOST@LOCALHOST","mapreduce.jobhistory.loadedjob.tasks.max":"-1","mapreduce.framework.name":"yarn","yarn.nodemanager.linux-container-executor.nonsecure-mode.user-pattern":"^[_.A-Za-z0-9][-@_.A-Za-z0-9]{0,255}?[$]?$","dfs.cachereport.intervalMsec":"10000","dfs.namenode.checkpoint.txns":"1000000","tfile.fs.output.buffer.size":"262144","yarn.app.mapreduce.am.job.task.listener.thread-count":"30","mapreduce.tasktracker.local.dir.minspacekill":"0","hadoop.security.groups.cache.background.reload.threads":"3","dfs.namenode.lease-recheck-interval-ms":"2000","fs.s3.block.size":"67108864","dfs.client.domain.socket.data.traffic":"false","dfs.ha.zkfc.nn.http.timeout.ms":"20000","hadoop.registry.secure":"false","hadoop.hdfs.configuration.version":"1","dfs.bytes-per-checksum":"512","fs.s3.buffer.dir":"${hadoop.tmp.dir}/s3","mapreduce.job.acl-view-job":" ","fs.s3a.s3guard.ddb.background.sleep":"25","mapreduce.jobhistory.loadedjobs.cache.size":"5","mapreduce.jobtracker.persist.jobstatus.hours":"1","fs.s3a.s3guard.ddb.table.create":"false","dfs.datanode.slow.io.warning.threshold.ms":"300","dfs.namenode.handler.count":"10","dfs.namenode.list.reencryption.status.num.responses":"100","mapreduce.input.fileinputformat.split.minsize":"0","dfs.datanode.failed.volumes.tolerated":"0","yarn.resourcemanager.container.liveness-monitor.interval-ms":"600000","yarn.resourcemanager.amliveliness-monitor.interval-ms":"1000","yarn.resourcemanager.client.thread-count":"50","io.seqfile.compress.blocksize":"1000000","mapreduce.tasktracker.http.threads":"40","dfs.namenode.retrycache.expirytime.millis":"600000","dfs.namenode.backup.address":"0.0.0.0:50100","dfs.datanode.data.dir":"file://${hadoop.tmp.dir}/dfs/data","dfs.datanode.shared.file.descriptor.paths":"/dev/shm,/tmp","dfs.replication":"3","mapreduce.jobtracker.jobhistory.block.size":"3145728","dfs.encrypt.data.transfer.cipher.key.bitlength":"128","mapreduce.reduce.shuffle.fetch.retry.interval-ms":"1000","dfs.secondary.namenode.kerberos.internal.spnego.principal":"${dfs.web.authentication.kerberos.principal}","mapreduce.task.profile.maps":"0-2","dfs.datanode.block-pinning.enabled":"false","yarn.nodemanager.admin-env":"MALLOC_ARENA_MAX=$MALLOC_ARENA_MAX","mapreduce.jobtracker.retiredjobs.cache.size":"1000","mapreduce.am.max-attempts":"2","hadoop.security.kms.client.failover.sleep.base.millis":"100","mapreduce.jobhistory.webapp.https.address":"test-1.vpc.company.com:19890","fs.trash.checkpoint.interval":"0","dfs.namenode.checkpoint.check.period":"60","yarn.nodemanager.container-monitor.interval-ms":"3000","mapreduce.job.map.output.collector.class":"org.apache.hadoop.mapred.MapTask$MapOutputBuffer","hadoop.http.authentication.signature.secret.file":"*********(redacted)","hadoop.jetty.logs.serve.aliases":"true","hadoop.proxyuser.HTTP.groups":"*","yarn.timeline-service.handler-thread-count":"10","yarn.resourcemanager.max-completed-applications":"10000","dfs.namenode.reencrypt.edek.threads":"10","yarn.resourcemanager.system-metrics-publisher.enabled":"false","hadoop.security.groups.negative-cache.secs":"30","yarn.app.mapreduce.task.container.log.backups":"0","hadoop.security.group.mapping.ldap.posix.attr.gid.name":"gidNumber","ipc.client.fallback-to-simple-auth-allowed":"false","dfs.namenode.fs-limits.max-component-length":"255","mapreduce.tasktracker.taskcontroller":"org.apache.hadoop.mapred.DefaultTaskController","yarn.client.failover-proxy-provider":"org.apache.hadoop.yarn.client.ConfiguredRMFailoverProxyProvider","yarn.timeline-service.http-authentication.simple.anonymous.allowed":"true","ha.health-monitor.check-interval.ms":"1000","dfs.namenode.top.window.num.buckets":"10","yarn.resourcemanager.store.class":"org.apache.hadoop.yarn.server.resourcemanager.recovery.FileSystemRMStateStore","dfs.datanode.block.id.layout.upgrade.threads":"12","mapreduce.jobtracker.tasktracker.maxblacklists":"4","yarn.nodemanager.docker-container-executor.exec-name":"/usr/bin/docker","yarn.resourcemanager.nodemanagers.heartbeat-interval-ms":"1000","hadoop.common.configuration.version":"0.23.0","fs.s3a.s3guard.ddb.table.capacity.read":"500","yarn.nodemanager.remote-app-log-dir-suffix":"logs","dfs.namenode.decommission.max.concurrent.tracked.nodes":"100","file.blocksize":"67108864","hadoop.registry.zk.retry.ceiling.ms":"60000","mapreduce.jobhistory.principal":"jhs/_HOST@REALM.TLD","dfs.client.read.shortcircuit.skip.checksum":"false","mapreduce.task.profile.reduces":"0-2","dfs.datanode.address":"0.0.0.0:50010","dfs.https.server.keystore.resource":"ssl-server.xml","yarn.timeline-service.webapp.https.address":"${yarn.timeline-service.hostname}:8190","yarn.resourcemanager.scheduler.address":"test-1.vpc.company.com:8030","mapreduce.task.timeout":"600000","hadoop.security.crypto.cipher.suite":"AES/CTR/NoPadding","yarn.resourcemanager.connect.max-wait.ms":"900000","fs.defaultFS":"hdfs://test-1.vpc.company.com:8020","fs.har.impl.disable.cache":"true","io.compression.codec.bzip2.library":"system-native","dfs.namenode.audit.loggers":"default","dfs.block.access.key.update.interval":"600","mapreduce.shuffle.connection-keep-alive.timeout":"5","yarn.resourcemanager.webapp.https.address":"test-1.vpc.company.com:8090","dfs.namenode.max.objects":"0","mapreduce.jobhistory.address":"test-1.vpc.company.com:10020","yarn.nodemanager.address":"${yarn.nodemanager.hostname}:0","fs.AbstractFileSystem.s3a.impl":"org.apache.hadoop.fs.s3a.S3A","mapreduce.task.combine.progress.records":"10000","dfs.namenode.max.full.block.report.leases":"6","yarn.resourcemanager.am.max-attempts":"2","yarn.nodemanager.linux-container-executor.cgroups.hierarchy":"/hadoop-yarn","dfs.client.mmap.cache.timeout.ms":"3600000","dfs.mover.max-no-move-interval":"60000","fs.ftp.transfer.mode":"BLOCK_TRANSFER_MODE","dfs.client.datanode-restart.timeout":"30","dfs.datanode.drop.cache.behind.reads":"false","ipc.server.log.slow.rpc":"false","dfs.namenode.read-lock-reporting-threshold-ms":"5000","yarn.app.mapreduce.am.job.committer.cancel-timeout":"60000","yarn.nodemanager.default-container-executor.log-dirs.permissions":"710","dfs.namenode.checkpoint.edits.dir":"${dfs.namenode.checkpoint.dir}","yarn.app.attempt.diagnostics.limit.kc":"64","dfs.balancer.block-move.timeout":"0","dfs.client.block.write.replace-datanode-on-failure.enable":"true","ftp.bytes-per-checksum":"512","yarn.nodemanager.resource.memory-mb":"8192","io.compression.codecs":"org.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.BZip2Codec,org.apache.hadoop.io.compress.DeflateCodec,org.apache.hadoop.io.compress.SnappyCodec,org.apache.hadoop.io.compress.Lz4Codec","fs.s3a.fast.upload.active.blocks":"4","hadoop.security.credential.clear-text-fallback":"true","dfs.heartbeat.interval":"3","mapreduce.jobhistory.joblist.cache.size":"20000","fs.ftp.host":"0.0.0.0","dfs.ha.tail-edits.period":"60","dfs.datanode.max.locked.memory":"0","dfs.datanode.scan.period.hours":"504","mapreduce.jobtracker.expire.trackers.interval":"600000","yarn.resourcemanager.nodemanager-connect-retries":"10","hadoop.security.kms.client.encrypted.key.cache.low-watermark":"0.3f","yarn.timeline-service.client.max-retries":"30","dfs.ha.fencing.ssh.connect-timeout":"30000","yarn.log-aggregation-enable":"false","mapreduce.reduce.markreset.buffer.percent":"0.0","fs.AbstractFileSystem.viewfs.impl":"org.apache.hadoop.fs.viewfs.ViewFs","dfs.namenode.edits.noeditlogchannelflush":"false","mapreduce.task.io.sort.factor":"64","mapreduce.tasktracker.outofband.heartbeat":"false","ha.failover-controller.new-active.rpc-timeout.ms":"60000","dfs.webhdfs.ugi.expire.after.access":"600000","mapreduce.jobhistory.datestring.cache.size":"200000","mapreduce.job.acl-modify-job":" ","dfs.namenode.https-address":"test-1.vpc.company.com:20102","yarn.am.blacklisting.enabled":"true","yarn.timeline-service.webapp.address":"${yarn.timeline-service.hostname}:8188","dfs.image.transfer-bootstrap-standby.bandwidthPerSec":"0","yarn.app.mapreduce.am.job.committer.commit-window":"10000","yarn.nodemanager.container-manager.thread-count":"20","yarn.timeline-service.ttl-enable":"true","mapreduce.jobhistory.recovery.store.fs.uri":"${hadoop.tmp.dir}/mapred/history/recoverystore","hadoop.proxyuser.hive.groups":"*","ha.zookeeper.session-timeout.ms":"5000","mapreduce.map.java.opts":"-Djava.net.preferIPv4Stack=true","tfile.io.chunk.size":"1048576","fs.s3a.s3guard.ddb.table.capacity.write":"100","mapreduce.job.speculative.slowtaskthreshold":"1.0","io.serializations":"org.apache.hadoop.io.serializer.WritableSerialization,org.apache.hadoop.io.serializer.avro.AvroSpecificSerialization,org.apache.hadoop.io.serializer.avro.AvroReflectSerialization","hadoop.security.kms.client.failover.sleep.max.millis":"2000","hadoop.security.group.mapping.ldap.directory.search.timeout":"10000","dfs.ha.automatic-failover.enabled":"false","mapreduce.job.counters.groups.max":"50","dfs.namenode.decommission.interval":"30","fs.swift.impl":"org.apache.hadoop.fs.swift.snative.SwiftNativeFileSystem","yarn.nodemanager.local-cache.max-files-per-directory":"8192","dfs.datanode.handler.count":"10","dfs.namenode.xattrs.enabled":"true","dfs.namenode.safemode.threshold-pct":"0.999f","dfs.client.socket.send.buffer.size":"0","mapreduce.map.sort.spill.percent":"0.8","yarn.resourcemanager.webapp.delegation-token-auth-filter.enabled":"*********(redacted)","hadoop.security.group.mapping.ldap.posix.attr.uid.name":"uidNumber","dfs.datanode.sync.behind.writes":"false","dfs.namenode.stale.datanode.interval":"30000","mapreduce.ifile.readahead":"true","yarn.timeline-service.leveldb-timeline-store.ttl-interval-ms":"300000","dfs.datanode.transfer.socket.send.buffer.size":"0","hadoop.security.kms.client.encrypted.key.cache.num.refill.threads":"2","dfs.namenode.reencrypt.throttle.limit.handler.ratio":"1.0","yarn.resourcemanager.scheduler.class":"org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler","yarn.app.mapreduce.am.command-opts":"-Djava.net.preferIPv4Stack=true -Xmx825955249","dfs.journalnode.https-address":"0.0.0.0:8481","mapreduce.cluster.local.dir":"${hadoop.tmp.dir}/mapred/local","hadoop.proxyuser.hue.hosts":"*","io.mapfile.bloom.error.rate":"0.005","dfs.user.home.dir.prefix":"/user","hadoop.proxyuser.hue.groups":"*","ha.failover-controller.graceful-fence.rpc-timeout.ms":"5000","ftp.replication":"3","mapreduce.jobtracker.persist.jobstatus.dir":"/jobtracker/jobsInfo","hadoop.security.uid.cache.secs":"14400","mapreduce.job.maxtaskfailures.per.tracker":"3","fs.s3a.metadatastore.impl":"org.apache.hadoop.fs.s3a.s3guard.NullMetadataStore","io.skip.checksum.errors":"false","dfs.namenode.snapshot.capture.openfiles":"false","dfs.datanode.directoryscan.interval":"21600","yarn.app.mapreduce.client-am.ipc.max-retries-on-timeouts":"3","dfs.client.read.shortcircuit.streams.cache.expiry.ms":"300000","fs.s3a.connection.timeout":"200000","mapreduce.job.max.split.locations":"10","dfs.namenode.write.stale.datanode.ratio":"0.5f","hadoop.registry.zk.session.timeout.ms":"60000","mapreduce.shuffle.transfer.buffer.size":"131072","yarn.timeline-service.client.retry-interval-ms":"1000","mapreduce.jobtracker.taskcache.levels":"2","yarn.http.policy":"HTTP_ONLY","fs.s3a.socket.send.buffer":"8192","hadoop.http.authentication.token.validity":"*********(redacted)","mapreduce.shuffle.max.connections":"0","mapreduce.job.emit-timeline-data":"false","hadoop.kerberos.min.seconds.before.relogin":"60","mapreduce.jobhistory.move.thread-count":"3","dfs.domain.socket.path":"/var/run/hdfs-sockets/dn","yarn.resourcemanager.admin.client.thread-count":"1","mapreduce.jobtracker.persist.jobstatus.active":"true","dfs.namenode.reencrypt.sleep.interval":"1m","fs.s3a.buffer.dir":"${hadoop.tmp.dir}/s3a","hadoop.ssl.enabled.protocols":"TLSv1,SSLv2Hello,TLSv1.1,TLSv1.2","mapreduce.jobhistory.admin.address":"test-1.vpc.company.com:10033","mapreduce.shuffle.port":"13562","yarn.resourcemanager.max-log-aggregation-diagnostics-in-memory":"10","yarn.nodemanager.health-checker.interval-ms":"600000","mapreduce.tasktracker.report.address":"127.0.0.1:0","dfs.namenode.edit.log.autoroll.multiplier.threshold":"2.0","io.seqfile.lazydecompress":"true","ftp.blocksize":"67108864","dfs.namenode.backup.http-address":"0.0.0.0:50105","dfs.disk.balancer.max.disk.throughputInMBperSec":"10","mapreduce.jobtracker.instrumentation":"org.apache.hadoop.mapred.JobTrackerMetricsInst","yarn.client.max-cached-nodemanagers-proxies":"0","yarn.nodemanager.delete.debug-delay-sec":"0","mapreduce.jobtracker.http.address":"0.0.0.0:50030","yarn.nodemanager.pmem-check-enabled":"true","yarn.nodemanager.disk-health-checker.max-disk-utilization-per-disk-percentage":"90.0","mapreduce.app-submission.cross-platform":"false","yarn.resourcemanager.work-preserving-recovery.scheduling-wait-ms":"10000","hadoop.security.groups.cache.secs":"300","yarn.resourcemanager.zk-retry-interval-ms":"1000","yarn.scheduler.increment-allocation-mb":"512","nfs.mountd.port":"4242","mapreduce.shuffle.max.threads":"0","hadoop.security.authorization":"false","mapreduce.job.complete.cancel.delegation.tokens":"*********(redacted)","fs.s3a.paging.maximum":"5000","nfs.exports.allowed.hosts":"* rw","mapreduce.jobhistory.http.policy":"HTTP_ONLY","dfs.datanode.dns.interface":"default","mapreduce.reduce.java.opts":"-Djava.net.preferIPv4Stack=true","s3native.replication":"3","hadoop.security.group.mapping.ldap.ssl":"false","dfs.namenode.fs-limits.max-xattrs-per-inode":"32","yarn.client.application-client-protocol.poll-interval-ms":"200","hadoop.proxyuser.flume.groups":"*","dfs.namenode.fs-limits.max-xattr-size":"16384","dfs.namenode.maintenance.replication.min":"1","dfs.client.write.exclude.nodes.cache.expiry.interval.millis":"600000","ha.zookeeper.parent-znode":"/hadoop-ha","dfs.namenode.safemode.extension":"30000","mapreduce.reduce.shuffle.merge.percent":"0.66","hadoop.security.group.mapping.ldap.search.filter.group":"(objectClass=group)","dfs.blocksize":"134217728","dfs.namenode.servicerpc-address":"test-1.vpc.company.com:8022","yarn.nodemanager.resourcemanager.minimum.version":"NONE","mapreduce.job.speculative.speculative-cap-running-tasks":"0.1","yarn.admin.acl":"*","yarn.resourcemanager.ha.automatic-failover.enabled":"true","mapreduce.reduce.skip.maxgroups":"0","mapreduce.reduce.shuffle.connect.timeout":"180000","yarn.resourcemanager.address":"test-1.vpc.company.com:8032","ipc.client.ping":"true","fs.adl.oauth2.access.token.provider.type":"*********(redacted)","dfs.namenode.resource.checked.volumes.minimum":"1","hadoop.proxyuser.HTTP.hosts":"*","mapreduce.shuffle.ssl.file.buffer.size":"65536","yarn.resourcemanager.ha.automatic-failover.embedded":"true","dfs.namenode.quota.init-threads":"4","dfs.journalnode.http-address":"0.0.0.0:8480","dfs.block.scanner.volume.bytes.per.second":"1048576","hadoop.ssl.enabled":"false","fs.s3a.multipart.purge":"false","dfs.storage.policy.enabled":"true","mapreduce.job.end-notification.max.attempts":"5","mapreduce.output.fileoutputformat.compress.codec":"org.apache.hadoop.io.compress.DefaultCodec","yarn.nodemanager.container-monitor.procfs-tree.smaps-based-rss.enabled":"false","dfs.namenode.edits.dir":"${dfs.namenode.name.dir}","ha.health-monitor.connect-retry-interval.ms":"1000","yarn.nodemanager.keytab":"/etc/krb5.keytab","dfs.namenode.support.allow.format":"true","dfs.ha.tail-edits.rolledits.timeout":"60","mapreduce.jobhistory.keytab":"/etc/security/keytab/jhs.service.keytab","fs.s3a.threads.max":"10","mapreduce.reduce.shuffle.input.buffer.percent":"0.70","mapreduce.cluster.temp.dir":"${hadoop.tmp.dir}/mapred/temp","s3.replication":"3","dfs.client.failover.connection.retries":"0","hadoop.tmp.dir":"/tmp/hadoop-${user.name}","mapreduce.job.maps":"2","dfs.namenode.secondary.http-address":"0.0.0.0:50090","mapreduce.job.end-notification.max.retry.interval":"5000","yarn.log-aggregation.retain-check-interval-seconds":"-1","yarn.resourcemanager.resource-tracker.client.thread-count":"50","nfs.wtmax":"1048576","yarn.timeline-service.leveldb-timeline-store.start-time-read-cache-size":"10000","nfs.dump.dir":"/tmp/.hdfs-nfs","yarn.resourcemanager.ha.automatic-failover.zk-base-path":"/yarn-leader-election","io.seqfile.local.dir":"${hadoop.tmp.dir}/io/local","mapreduce.client.submit.file.replication":"3","mapreduce.jobhistory.minicluster.fixed.ports":"false","fs.s3a.multipart.threshold":"128M","dfs.namenode.service.handler.count":"10","dfs.datanode.data.dir.perm":"700","mapreduce.jobhistory.done-dir":"${yarn.app.mapreduce.am.staging-dir}/history/done","dfs.namenode.name.dir":"file:///dataroot/dataroot/dfs/nn","yarn.resourcemanager.zk-acl":"world:anyone:rwcda","ipc.client.idlethreshold":"4000","yarn.nodemanager.linux-container-executor.cgroups.strict-resource-usage":"false","mapreduce.reduce.input.buffer.percent":"0.0","fs.ftp.host.port":"21","ipc.ping.interval":"60000","dfs.namenode.num.checkpoints.retained":"2","dfs.namenode.kerberos.internal.spnego.principal":"${dfs.web.authentication.kerberos.principal}","yarn.resourcemanager.admin.address":"test-1.vpc.company.com:8033","file.client-write-packet-size":"65536","hadoop.treat.subject.external":"false","ipc.client.kill.max":"10","mapreduce.reduce.speculative":"false","dfs.disk.balancer.plan.threshold.percent":"10","mapreduce.local.clientfactory.class.name":"org.apache.hadoop.mapred.LocalClientFactory","dfs.client.use.legacy.blockreader":"false","mapreduce.job.reducer.unconditional-preempt.delay.sec":"300","yarn.nodemanager.disk-health-checker.interval-ms":"120000","ipc.client.connection.maxidletime":"10000","mapreduce.task.io.sort.mb":"256","yarn.nodemanager.localizer.client.thread-count":"5","dfs.namenode.checkpoint.max-retries":"3","dfs.namenode.reject-unresolved-dn-topology-mapping":"false","dfs.namenode.delegation.token.max-lifetime":"*********(redacted)","dfs.client.block.write.replace-datanode-on-failure.min-replication":"0","yarn.nodemanager.localizer.cache.cleanup.interval-ms":"600000","hadoop.security.crypto.codec.classes.aes.ctr.nopadding":"org.apache.hadoop.crypto.OpensslAesCtrCryptoCodec,org.apache.hadoop.crypto.JceAesCtrCryptoCodec","fs.s3a.connection.ssl.enabled":"true","yarn.nodemanager.process-kill-wait.ms":"2000","dfs.namenode.num.extra.edits.retained":"1000000","mapreduce.job.hdfs-servers":"${fs.defaultFS}","yarn.scheduler.increment-allocation-vcores":"1","fs.df.interval":"60000","fs.s3.sleepTimeSeconds":"10","fs.s3a.multiobjectdelete.enable":"true","yarn.nodemanager.disk-health-checker.min-healthy-disks":"0.25","hadoop.shell.missing.defaultFs.warning":"true","io.file.buffer.size":"65536","hadoop.work.around.non.threadsafe.getpwuid":"false","dfs.permissions.superusergroup":"supergroup","hadoop.security.group.mapping.ldap.search.attr.member":"member","hadoop.security.random.device.file.path":"/dev/urandom","mapreduce.tasktracker.dns.interface":"default","hadoop.security.sensitive-config-keys":"*********(redacted)","fs.s3a.s3guard.ddb.max.retries":"9","hadoop.rpc.socket.factory.class.default":"org.apache.hadoop.net.StandardSocketFactory","dfs.permissions.enabled":"true","yarn.resourcemanager.connect.retry-interval.ms":"30000","yarn.scheduler.minimum-allocation-mb":"1024","yarn.app.mapreduce.am.staging-dir":"/user","mapreduce.reduce.shuffle.read.timeout":"180000","yarn.app.mapreduce.am.admin.user.env":"LD_LIBRARY_PATH=$HADOOP_COMMON_HOME/lib/native:$JAVA_LIBRARY_PATH","dfs.datanode.https.address":"0.0.0.0:50475","dfs.namenode.hosts.provider.classname":"org.apache.hadoop.hdfs.server.blockmanagement.HostFileManager","dfs.datanode.transfer.socket.recv.buffer.size":"0","fs.s3a.connection.establish.timeout":"5000","dfs.namenode.fslock.fair":"true","mapreduce.job.running.map.limit":"0","hadoop.ssl.require.client.cert":"false","hadoop.kerberos.kinit.command":"kinit","hadoop.fuse.connection.timeout":"300","mapreduce.reduce.log.level":"INFO","hadoop.security.dns.log-slow-lookups.threshold.ms":"1000","mapreduce.job.ubertask.enable":"false","adl.http.timeout":"5000","yarn.nodemanager.vmem-pmem-ratio":"2.1","dfs.client.slow.io.warning.threshold.ms":"30000","hadoop.rpc.protection":"authentication","ha.health-monitor.rpc-timeout.ms":"45000","s3native.stream-buffer-size":"4096","yarn.nodemanager.remote-app-log-dir":"/tmp/logs","fs.s3a.s3guard.cli.prune.age":"86400000","dfs.client.read.shortcircuit.streams.cache.size":"256","dfs.client.use.legacy.blockreader.local":"false","yarn.app.mapreduce.am.containerlauncher.threadpool-initial-size":"10","fs.s3n.multipart.uploads.enabled":"false","dfs.namenode.path.based.cache.retry.interval.ms":"30000","hadoop.security.crypto.buffer.size":"8192","yarn.client.failover-retries-on-socket-timeouts":"0","dfs.balancer.keytab.enabled":"false","hadoop.security.instrumentation.requires.admin":"false","yarn.nodemanager.delete.thread-count":"4","dfs.datanode.balance.bandwidthPerSec":"10485760","dfs.namenode.name.dir.restore":"false","hadoop.registry.jaas.context":"Client","dfs.client.failover.sleep.max.millis":"15000","yarn.timeline-service.leveldb-timeline-store.path":"${hadoop.tmp.dir}/yarn/timeline","s3.blocksize":"67108864","yarn.am.blacklisting.disable-failure-threshold":"0.8f","io.map.index.interval":"128","mapreduce.job.counters.max":"120","dfs.namenode.max-lock-hold-to-release-lease-ms":"25","dfs.namenode.datanode.registration.ip-hostname-check":"true","yarn.timeline-service.store-class":"org.apache.hadoop.yarn.server.timeline.LeveldbTimelineStore","mapreduce.jobhistory.move.interval-ms":"180000","dfs.namenode.resource.du.reserved":"104857600","dfs.datanode.bp-ready.timeout":"20","yarn.nodemanager.localizer.fetch.thread-count":"4","yarn.resourcemanager.scheduler.client.thread-count":"50","hadoop.ssl.hostname.verifier":"DEFAULT","dfs.namenode.full.block.report.lease.length.ms":"300000","mapreduce.tasktracker.instrumentation":"org.apache.hadoop.mapred.TaskTrackerMetricsInst","mapreduce.job.classloader":"false","mapreduce.task.profile.map.params":"${mapreduce.task.profile.params}","ipc.client.connect.timeout":"20000","s3.stream-buffer-size":"4096","yarn.resourcemanager.nm.liveness-monitor.interval-ms":"1000","yarn.nm.liveness-monitor.expiry-interval-ms":"600000","dfs.namenode.secondary.https-address":"0.0.0.0:50091","s3native.bytes-per-checksum":"512","dfs.namenode.fs-limits.max-directory-items":"1048576","nfs.server.port":"2049","dfs.namenode.delegation.token.renew-interval":"*********(redacted)","mapreduce.jobtracker.address":"local","yarn.nodemanager.recovery.enabled":"false","mapreduce.job.end-notification.retry.interval":"1000","fs.du.interval":"600000","dfs.namenode.list.openfiles.num.responses":"1000","hadoop.security.groups.cache.warn.after.ms":"5000","file.bytes-per-checksum":"512","dfs.namenode.blocks.per.postponedblocks.rescan":"10000","dfs.namenode.checkpoint.period":"3600","hadoop.security.groups.cache.background.reload":"false","yarn.resourcemanager.amlauncher.log.command":"false","net.topology.script.number.args":"100","mapreduce.task.merge.progress.records":"10000","yarn.nodemanager.localizer.address":"${yarn.nodemanager.hostname}:8040","yarn.timeline-service.keytab":"/etc/krb5.keytab","mapreduce.reduce.shuffle.fetch.retry.timeout-ms":"30000","dfs.namenode.snapshot.skip.capture.accesstime-only-change":"false","dfs.webhdfs.user.provider.user.pattern":"^[A-Za-z_][A-Za-z0-9._-]*[$]?$","dfs.webhdfs.acl.provider.permission.pattern":"^(default:)?(user|group|mask|other):[[A-Za-z_][A-Za-z0-9._-]]*:([rwx-]{3})?(,(default:)?(user|group|mask|other):[[A-Za-z_][A-Za-z0-9._-]]*:([rwx-]{3})?)*$","mapreduce.fileoutputcommitter.algorithm.version":"1","yarn.resourcemanager.work-preserving-recovery.enabled":"false","mapreduce.map.skip.maxrecords":"0","mapreduce.jobtracker.handler.count":"10","hadoop.http.authentication.type":"simple","mapreduce.job.jvm.numtasks":"1","hadoop.proxyuser.flume.hosts":"*","mapreduce.task.userlog.limit.kb":"0","yarn.resourcemanager.scheduler.monitor.enable":"false","fs.s3n.block.size":"67108864","ipc.client.connect.max.retries":"10","hadoop.registry.zk.retry.times":"5","mapreduce.jobtracker.staging.root.dir":"${hadoop.tmp.dir}/mapred/staging","dfs.namenode.http-address":"test-1.vpc.company.com:20101","mapreduce.jobtracker.jobhistory.lru.cache.size":"5","dfs.datanode.directoryscan.threads":"1","dfs.datanode.fsdatasetcache.max.threads.per.volume":"4","dfs.namenode.fs-limits.max-blocks-per-file":"1048576","dfs.disk.balancer.enabled":"false","mapreduce.shuffle.listen.queue.size":"128","mapreduce.tasktracker.local.dir.minspacestart":"0","mapreduce.map.cpu.vcores":"1","hadoop.user.group.static.mapping.overrides":"dr.who=;","dfs.datanode.cache.revocation.timeout.ms":"900000","mapreduce.jobhistory.recovery.store.class":"org.apache.hadoop.mapreduce.v2.hs.HistoryServerFileSystemStateStoreService","dfs.client.mmap.cache.size":"256","dfs.ha.log-roll.period":"120","dfs.client.failover.sleep.base.millis":"500","yarn.resourcemanager.fail-fast":"${yarn.fail-fast}","yarn.resourcemanager.proxy-user-privileges.enabled":"false","mapreduce.job.reducer.preempt.delay.sec":"0","hadoop.util.hash.type":"murmur","dfs.namenode.accesstime.precision":"3600000","yarn.app.mapreduce.client.job.max-retries":"3","mapreduce.reduce.shuffle.retry-delay.max.ms":"60000","mapreduce.task.profile.params":"-agentlib:hprof=cpu=samples,heap=sites,force=n,thread=y,verbose=n,file=%s","yarn.app.mapreduce.shuffle.log.backups":"0","hadoop.registry.zk.retry.interval.ms":"1000","fs.AbstractFileSystem.file.impl":"org.apache.hadoop.fs.local.LocalFs","yarn.nodemanager.log-aggregation.roll-monitoring-interval-seconds":"-1","dfs.client.context":"default","mapreduce.jobhistory.cleaner.interval-ms":"86400000","hadoop.registry.zk.quorum":"localhost:2181","mapreduce.output.fileoutputformat.compress":"false","yarn.resourcemanager.am-rm-tokens.master-key-rolling-interval-secs":"*********(redacted)","hadoop.ssl.server.conf":"ssl-server.xml","dfs.http.policy":"HTTP_ONLY","dfs.client.https.keystore.resource":"ssl-client.xml","mapreduce.client.completion.pollinterval":"5000","hadoop.ssl.keystores.factory.class":"org.apache.hadoop.security.ssl.FileBasedKeyStoresFactory","yarn.app.mapreduce.am.resource.cpu-vcores":"1","yarn.timeline-service.enabled":"false","yarn.acl.enable":"true","dfs.domain.socket.disable.interval.seconds":"1","dfs.image.transfer.chunksize":"65536","dfs.balancer.max-no-move-interval":"60000","mapreduce.tasktracker.map.tasks.maximum":"2","dfs.namenode.edits.journal-plugin.qjournal":"org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager","mapreduce.task.profile":"false","dfs.webhdfs.enabled":"true","yarn.resourcemanager.fs.state-store.uri":"${hadoop.tmp.dir}/yarn/system/rmstore","yarn.nodemanager.linux-container-executor.nonsecure-mode.local-user":"nobody","dfs.namenode.list.encryption.zones.num.responses":"100","yarn.resourcemanager.configuration.provider-class":"org.apache.hadoop.yarn.LocalConfigurationProvider","dfs.namenode.top.num.users":"10","dfs.disk.balancer.block.tolerance.percent":"10","yarn.nodemanager.resource.percentage-physical-cpu-limit":"100","mapreduce.jobhistory.client.thread-count":"10","tfile.fs.input.buffer.size":"262144","mapreduce.client.progressmonitor.pollinterval":"1000","yarn.nodemanager.log-dirs":"${yarn.log.dir}/userlogs","io.seqfile.sorter.recordlimit":"1000000","hadoop.security.auth_to_local":"DEFAULT","dfs.blockreport.initialDelay":"0","fs.automatic.close":"true","dfs.client.block.write.replace-datanode-on-failure.best-effort":"false","dfs.namenode.replication.min":"1","dfs.balancer.address":"0.0.0.0:0","fs.s3n.multipart.copy.block.size":"5368709120","yarn.nodemanager.hostname":"0.0.0.0","nfs.rtmax":"1048576","yarn.resourcemanager.zk-timeout-ms":"10000","ftp.stream-buffer-size":"4096","yarn.fail-fast":"false","hadoop.security.group.mapping.ldap.search.filter.user":"(&(objectClass=user)(sAMAccountName={0}))","dfs.datanode.directoryscan.throttle.limit.ms.per.sec":"1000","yarn.nodemanager.container-localizer.log.level":"INFO","yarn.timeline-service.address":"${yarn.timeline-service.hostname}:10200","dfs.namenode.replication.work.multiplier.per.iteration":"2","mapreduce.job.ubertask.maxmaps":"9","fs.s3a.threads.keepalivetime":"60","dfs.namenode.reencrypt.throttle.limit.updater.ratio":"1.0","dfs.namenode.avoid.write.stale.datanode":"false","dfs.short.circuit.shared.memory.watcher.interrupt.check.ms":"60000","dfs.datanode.available-space-volume-choosing-policy.balanced-space-preference-fraction":"0.75f","mapreduce.task.files.preserve.failedtasks":"false","yarn.app.mapreduce.client.job.retry-interval":"2000","ha.failover-controller.graceful-fence.connection.retries":"1","dfs.client.mmap.enabled":"true","mapreduce.reduce.cpu.vcores":"1","hadoop.proxyuser.oozie.groups":"*","fs.client.resolve.remote.symlinks":"true","dfs.image.compression.codec":"org.apache.hadoop.io.compress.DefaultCodec","mapreduce.jobtracker.restart.recover":"false","dfs.namenode.decommission.blocks.per.interval":"500000","mapreduce.tasktracker.reduce.tasks.maximum":"2","yarn.nodemanager.local-dirs":"${hadoop.tmp.dir}/nm-local-dir","mapreduce.shuffle.connection-keep-alive.enable":"false","fs.s3a.path.style.access":"false","yarn.nodemanager.aux-services.mapreduce_shuffle.class":"org.apache.hadoop.mapred.ShuffleHandler","fs.adl.impl":"org.apache.hadoop.fs.adl.AdlFileSystem","yarn.resourcemanager.nodemanager.minimum.version":"NONE","net.topology.impl":"org.apache.hadoop.net.NetworkTopology","io.map.index.skip":"0","dfs.namenode.safemode.min.datanodes":"0","fs.ftp.data.connection.mode":"ACTIVE_LOCAL_DATA_CONNECTION_MODE","mapreduce.job.userlog.retain.hours":"24","yarn.scheduler.maximum-allocation-vcores":"4","yarn.nodemanager.log-aggregation.compression-type":"none","dfs.namenode.enable.retrycache":"true","yarn.ipc.rpc.class":"org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC","dfs.namenode.startup.delay.block.deletion.sec":"0","mapreduce.reduce.maxattempts":"4","hadoop.security.dns.log-slow-lookups.enabled":"false","mapreduce.job.committer.setup.cleanup.needed":"true","dfs.datanode.readahead.bytes":"4194304","mapreduce.jobtracker.heartbeats.in.second":"100","mapreduce.job.running.reduce.limit":"0","mapreduce.job.token.tracking.ids.enabled":"*********(redacted)","mapreduce.task.tmp.dir":"./tmp","hadoop.registry.system.acls":"sasl:yarn@, sasl:mapred@, sasl:mapred@hdfs@","yarn.nodemanager.recovery.dir":"${hadoop.tmp.dir}/yarn-nm-recovery","fs.s3a.fast.upload.buffer":"disk","mapreduce.jobhistory.intermediate-done-dir":"${yarn.app.mapreduce.am.staging-dir}/history/done_intermediate","yarn.app.mapreduce.shuffle.log.separate":"true","dfs.namenode.delegation.key.update-interval":"86400000","fs.s3a.max.total.tasks":"5","dfs.client.file-block-storage-locations.num-threads":"10","mapreduce.tasktracker.healthchecker.interval":"60000","fs.s3a.readahead.range":"64K","hadoop.http.authentication.simple.anonymous.allowed":"true","fs.s3a.fast.upload":"false","fs.s3a.attempts.maximum":"20","dfs.namenode.avoid.read.stale.datanode":"false","hadoop.registry.zk.connection.timeout.ms":"15000","dfs.https.port":"20102","yarn.nodemanager.health-checker.script.timeout-ms":"1200000","yarn.timeline-service.leveldb-timeline-store.start-time-write-cache-size":"10000","mapreduce.map.log.level":"INFO","mapreduce.output.fileoutputformat.compress.type":"BLOCK","hadoop.registry.rm.enabled":"false","mapreduce.ifile.readahead.bytes":"4194304","mapreduce.tasktracker.tasks.sleeptimebeforesigkill":"5000","yarn.resourcemanager.fs.state-store.retry-policy-spec":"2000, 500","dfs.namenode.posix.acl.inheritance.enabled":"false","dfs.blockreport.intervalMsec":"21600000","yarn.nodemanager.linux-container-executor.nonsecure-mode.limit-users":"true","mapreduce.cluster.acls.enabled":"false","mapreduce.job.speculative.retry-after-no-speculate":"1000","dfs.namenode.path.based.cache.refresh.interval.ms":"30000","dfs.namenode.edekcacheloader.interval.ms":"1000","file.stream-buffer-size":"4096","mapreduce.map.output.compress.codec":"org.apache.hadoop.io.compress.SnappyCodec","mapreduce.map.speculative":"false","dfs.disk.balancer.max.disk.errors":"5","dfs.datanode.use.datanode.hostname":"false","mapreduce.job.speculative.retry-after-speculate":"15000","hadoop.proxyuser.hdfs.hosts":"*","dfs.namenode.fs-limits.min-block-size":"1048576","yarn.nodemanager.linux-container-executor.cgroups.mount":"false","yarn.app.mapreduce.am.container.log.backups":"0","mapreduce.job.reduce.slowstart.completedmaps":"0.8","dfs.client.read.shortcircuit":"false","yarn.timeline-service.http-authentication.type":"simple","hadoop.security.group.mapping.ldap.search.attr.group.name":"cn","hadoop.proxyuser.yarn.groups":"*","dfs.client.cached.conn.retry":"3","dfs.namenode.invalidate.work.pct.per.iteration":"0.32f","hadoop.http.logs.enabled":"true","fs.s3a.block.size":"32M","yarn.nodemanager.logaggregation.threadpool-size-max":"100","dfs.replication.max":"512","dfs.namenode.inotify.max.events.per.rpc":"1000","yarn.resourcemanager.hostname":"0.0.0.0","mapreduce.reduce.shuffle.fetch.retry.enabled":"${yarn.nodemanager.recovery.enabled}","mapreduce.map.memory.mb":"0","mapreduce.task.skip.start.attempts":"2","fs.AbstractFileSystem.hdfs.impl":"org.apache.hadoop.fs.Hdfs","ipc.client.rpc-timeout.ms":"0","fs.s3.maxRetries":"4","dfs.default.chunk.view.size":"32768","mapreduce.input.lineinputformat.linespermap":"1","ipc.client.connect.max.retries.on.timeouts":"45","yarn.timeline-service.leveldb-timeline-store.read-cache-size":"104857600","fs.AbstractFileSystem.har.impl":"org.apache.hadoop.fs.HarFs","mapreduce.job.split.metainfo.maxsize":"10000000","yarn.am.liveness-monitor.expiry-interval-ms":"600000","dfs.client.mmap.retry.timeout.ms":"300000","yarn.resourcemanager.container-tokens.master-key-rolling-interval-secs":"*********(redacted)","dfs.namenode.list.cache.directives.num.responses":"100","fs.s3a.socket.recv.buffer":"8192","dfs.image.compress":"false","dfs.namenode.kerberos.principal.pattern":"*","yarn.application.classpath":"$HADOOP_CLIENT_CONF_DIR,$HADOOP_CONF_DIR,$HADOOP_COMMON_HOME/*,$HADOOP_COMMON_HOME/lib/*,$HADOOP_HDFS_HOME/*,$HADOOP_HDFS_HOME/lib/*,$HADOOP_YARN_HOME/*,$HADOOP_YARN_HOME/lib/*","fs.s3n.multipart.uploads.block.size":"67108864","mapreduce.tasktracker.http.address":"0.0.0.0:50060","yarn.resourcemanager.resource-tracker.address":"test-1.vpc.company.com:8031","hadoop.fuse.timer.period":"5","mapreduce.job.heap.memory-mb.ratio":"0.8","dfs.datanode.hdfs-blocks-metadata.enabled":"true","dfs.namenode.checkpoint.dir":"file://${hadoop.tmp.dir}/dfs/namesecondary","dfs.datanode.max.transfer.threads":"4096","dfs.namenode.edits.asynclogging":"true","nfs.allow.insecure.ports":"true","mapreduce.client.output.filter":"FAILED","hadoop.http.filter.initializers":"org.apache.hadoop.http.lib.StaticUserWebFilter","mapreduce.reduce.memory.mb":"0","s3native.client-write-packet-size":"65536","mapreduce.admin.user.env":"LD_LIBRARY_PATH=$HADOOP_COMMON_HOME/lib/native:$JAVA_LIBRARY_PATH","yarn.timeline-service.hostname":"0.0.0.0","file.replication":"1","yarn.nodemanager.container-metrics.unregister-delay-ms":"10000","hadoop.proxyuser.mapred.hosts":"*","hadoop.proxyuser.oozie.hosts":"*","yarn.nodemanager.log.retain-seconds":"10800","hadoop.proxyuser.mapred.groups":"*","yarn.resourcemanager.keytab":"/etc/krb5.keytab","mapreduce.reduce.merge.inmem.threshold":"1000","dfs.client.https.need-auth":"false","dfs.blockreport.split.threshold":"1000000","dfs.client.block.write.replace-datanode-on-failure.policy":"DEFAULT","mapreduce.shuffle.ssl.enabled":"false","dfs.namenode.write-lock-reporting-threshold-ms":"5000","dfs.block.access.token.enable":"*********(redacted)","yarn.resourcemanager.state-store.max-completed-applications":"${yarn.resourcemanager.max-completed-applications}","httpfs.buffer.size":"4096","dfs.client.file-block-storage-locations.timeout.millis":"1000","dfs.namenode.block-placement-policy.default.prefer-local-node":"true","mapreduce.job.speculative.minimum-allowed-tasks":"10","yarn.log-aggregation.retain-seconds":"-1","dfs.namenode.replication.considerLoad":"true","yarn.nodemanager.disk-health-checker.min-free-space-per-disk-mb":"0","mapreduce.jobhistory.max-age-ms":"604800000","hadoop.proxyuser.hdfs.groups":"*","dfs.namenode.retrycache.heap.percent":"0.03f","dfs.datanode.cache.revocation.polling.ms":"500","mapreduce.jobhistory.webapp.address":"test-1.vpc.company.com:19888","dfs.namenode.path.based.cache.block.map.allocation.percent":"0.25","mapreduce.jobtracker.system.dir":"${hadoop.tmp.dir}/mapred/system","mapreduce.tasktracker.taskmemorymanager.monitoringinterval":"5000","dfs.journalnode.rpc-address":"0.0.0.0:8485","yarn.client.nodemanager-connect.max-wait-ms":"180000","yarn.resourcemanager.webapp.address":"test-1.vpc.company.com:8088","mapreduce.jobhistory.recovery.enable":"false","dfs.client.short.circuit.replica.stale.threshold.ms":"1800000","mapreduce.reduce.shuffle.parallelcopies":"10","fs.trash.interval":"1","dfs.namenode.replication.interval":"3","yarn.app.mapreduce.client.max-retries":"3","hadoop.security.authentication":"simple","dfs.namenode.top.enabled":"true","mapreduce.task.profile.reduce.params":"${mapreduce.task.profile.params}","dfs.datanode.du.reserved":"0","yarn.app.mapreduce.am.resource.mb":"1024","mapreduce.input.fileinputformat.list-status.num-threads":"1","dfs.namenode.lazypersist.file.scrub.interval.sec":"300","yarn.nodemanager.container-executor.class":"org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor","io.mapfile.bloom.size":"1048576","yarn.timeline-service.ttl-ms":"604800000","yarn.nodemanager.resource.cpu-vcores":"8","mapreduce.job.reduces":"6","fs.s3a.multipart.size":"64M","yarn.scheduler.minimum-allocation-vcores":"1","dfs.namenode.reencrypt.batch.size":"1000","mapreduce.job.speculative.speculative-cap-total-tasks":"0.01","dfs.datanode.http.address":"0.0.0.0:50075","hadoop.ssl.client.conf":"ssl-client.xml","mapreduce.job.queuename":"default","fs.s3a.metadatastore.authoritative":"false","ha.health-monitor.sleep-after-disconnect.ms":"1000","s3.bytes-per-checksum":"512","yarn.app.mapreduce.shuffle.log.limit.kb":"0","dfs.namenode.list.cache.pools.num.responses":"100","hadoop.security.group.mapping":"org.apache.hadoop.security.ShellBasedUnixGroupsMapping","mapreduce.jobhistory.jhist.format":"binary","yarn.resourcemanager.ha.enabled":"false","dfs.encrypt.data.transfer":"false","hadoop.http.staticuser.user":"dr.who","mapreduce.task.exit.timeout.check-interval-ms":"20000","mapreduce.task.exit.timeout":"60000","yarn.nodemanager.linux-container-executor.resources-handler.class":"org.apache.hadoop.yarn.server.nodemanager.util.DefaultLCEResourcesHandler","mapreduce.reduce.shuffle.memory.limit.percent":"0.25","mapreduce.job.redacted-properties":"*********(redacted)","dfs.namenode.top.windows.minutes":"1,5,25","s3.client-write-packet-size":"65536","mapreduce.map.output.compress":"true","ha.zookeeper.acl":"world:anyone:rwcda","ipc.server.max.connections":"0","yarn.scheduler.maximum-allocation-mb":"12288","yarn.resourcemanager.scheduler.monitor.policies":"org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy","yarn.app.mapreduce.am.container.log.limit.kb":"0","s3native.blocksize":"67108864","ipc.client.connect.retry.interval":"1000","hadoop.proxyuser.httpfs.groups":"*","yarn.resourcemanager.zk-state-store.parent-path":"/rmstore","dfs.namenode.edit.log.autoroll.check.interval.ms":"300000","mapreduce.jobhistory.cleaner.enable":"true","hadoop.security.kms.client.encrypted.key.cache.expiry":"43200000","hadoop.proxyuser.httpfs.hosts":"*","dfs.client.use.datanode.hostname":"false","dfs.stream-buffer-size":"4096","yarn.client.nodemanager-client-async.thread-pool-max-size":"500","mapreduce.map.maxattempts":"4","dfs.datanode.drop.cache.behind.writes":"false","mapreduce.tasktracker.dns.nameserver":"default","yarn.nodemanager.sleep-delay-before-sigkill.ms":"250","mapreduce.job.end-notification.retry.attempts":"0","hadoop.proxyuser.yarn.hosts":"*","yarn.resourcemanager.zk-num-retries":"1000","dfs.client.failover.max.attempts":"15","mapreduce.tasktracker.indexcache.mb":"10","hadoop.registry.zk.root":"/registry","adl.feature.ownerandgroup.enableupn":"false","mapreduce.job.reduce.shuffle.consumer.plugin.class":"org.apache.hadoop.mapreduce.task.reduce.Shuffle","yarn.resourcemanager.delayed.delegation-token.removal-interval-ms":"*********(redacted)","dfs.namenode.snapshotdiff.allow.snap-root-descendant":"true","yarn.nodemanager.localizer.cache.target-size-mb":"10240","zlib.compress.level":"DEFAULT_COMPRESSION","ftp.client-write-packet-size":"65536","mapreduce.jobtracker.maxtasks.perjob":"-1","fs.AbstractFileSystem.adl.impl":"org.apache.hadoop.fs.adl.Adl","hadoop.proxyuser.hive.hosts":"*","dfs.block.access.token.lifetime":"*********(redacted)","dfs.namenode.max.extra.edits.segments.retained":"10000","yarn.client.failover-retries":"0","fs.s3a.multipart.purge.age":"86400","dfs.image.transfer.bandwidthPerSec":"0","io.native.lib.available":"true","net.topology.node.switch.mapping.impl":"org.apache.hadoop.net.ScriptBasedMapping","ipc.server.listen.queue.size":"128","dfs.namenode.edekcacheloader.initial.delay.ms":"3000","map.sort.class":"org.apache.hadoop.util.QuickSort","dfs.namenode.acls.enabled":"false","hadoop.security.kms.client.authentication.retry-count":"1","fs.permissions.umask-mode":"022","dfs.datanode.ipc.address":"0.0.0.0:50020","yarn.nodemanager.vmem-check-enabled":"false","yarn.nodemanager.recovery.compaction-interval-secs":"3600","yarn.app.mapreduce.client-am.ipc.max-retries":"3","dfs.lock.suppress.warning.interval":"10s","dfs.client.block.write.retries":"3","mapreduce.job.ubertask.maxreduces":"1","hadoop.security.kms.client.encrypted.key.cache.size":"500","hadoop.security.java.secure.random.algorithm":"SHA1PRNG","ha.failover-controller.cli-check.rpc-timeout.ms":"20000","mapreduce.application.classpath":"$HADOOP_MAPRED_HOME/*,$HADOOP_MAPRED_HOME/lib/*,$MR2_CLASSPATH","yarn.client.nodemanager-connect.retry-interval-ms":"10000","dfs.client-write-packet-size":"65536","yarn.nodemanager.env-whitelist":"JAVA_HOME,HADOOP_COMMON_HOME,HADOOP_HDFS_HOME,HADOOP_CONF_DIR,HADOOP_YARN_HOME","dfs.datanode.dns.nameserver":"default","yarn.nodemanager.webapp.address":"${yarn.nodemanager.hostname}:8042","rpc.metrics.quantile.enable":"false","mapreduce.jobhistory.admin.acl":"*","yarn.resourcemanager.system-metrics-publisher.dispatcher.pool-size":"10","hadoop.http.authentication.kerberos.keytab":"${user.home}/hadoop.keytab","dfs.image.transfer.timeout":"60000","yarn.resourcemanager.recovery.enabled":"false","dfs.datanode.available-space-volume-choosing-policy.balanced-space-threshold":"10737418240","dfs.client.failover.connection.retries.on.timeouts":"0"},"System Properties":{"java.io.tmpdir":"/tmp","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.security.egd":"file:///dev/urandom","java.vm.specification.version":"1.8","user.home":"/home/systest","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/usr/java/jdk1.8.0_144/jre/lib/amd64","user.dir":"/tmp","java.library.path":":/opt/cloudera/parcels/CDH/lib/hadoop/lib/native:/opt/cloudera/parcels/CDH/lib/hadoop/lib/native:/opt/cloudera/parcels/CDH/lib/hadoop/lib/native:/usr/java/packages/lib/amd64:/usr/lib64:/lib64:/lib:/usr/lib","sun.cpu.isalist":"","os.arch":"amd64","java.vm.version":"25.144-b01","jetty.git.hash":"27208684755d94a92186989f695db2d7b21ebc51","java.endorsed.dirs":"/usr/java/jdk1.8.0_144/jre/lib/endorsed","java.runtime.version":"1.8.0_144-b01","java.vm.info":"mixed mode","java.ext.dirs":"/usr/java/jdk1.8.0_144/jre/lib/ext:/usr/java/packages/lib/ext","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"52.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/usr/java/jdk1.8.0_144/jre/lib/resources.jar:/usr/java/jdk1.8.0_144/jre/lib/rt.jar:/usr/java/jdk1.8.0_144/jre/lib/sunrsasign.jar:/usr/java/jdk1.8.0_144/jre/lib/jsse.jar:/usr/java/jdk1.8.0_144/jre/lib/jce.jar:/usr/java/jdk1.8.0_144/jre/lib/charsets.jar:/usr/java/jdk1.8.0_144/jre/lib/jfr.jar:/usr/java/jdk1.8.0_144/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Los_Angeles","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"3.10.0-514.26.2.el7.x86_64","sun.os.patch.level":"unknown","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","user.language":"en","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.print.PSPrinterJob","java.awt.graphicsenv":"sun.awt.X11GraphicsEnvironment","awt.toolkit":"sun.awt.X11.XToolkit","os.name":"Linux","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"systest","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master yarn --deploy-mode client --conf spark.driver.memory=2g --conf spark.executor.heartbeatInterval=1000 --conf spark.executor.metrics.pollingInterval=100 --conf spark.eventLog.logStageExecutorProcessTreeMetrics.enabled=true --conf spark.yarn.maxAppAttempts=1 --conf spark.locality.wait.process=0 --conf spark.executor.memoryOverhead=1024 --conf spark.executor.extraJavaOptions=-Djava.security.egd=file:///dev/urandom --conf spark.eventLog.logStageExecutorMetrics=true --conf spark.driver.extraJavaOptions=-Djava.security.egd=file:///dev/urandom --class com.company.spark.LargeBlocks --num-executors 3 --executor-memory 7g /tmp/__spark_test__/spark3-tests-0.1.0-cdh5.9.0-SNAPSHOT-jar-with-dependencies.jar --targetBlockSizeGb 2.5 --taskSleepMillis 200 --doCache true --cacheOnDisk true --replicas 1 --concurrentReadJobs 2","java.home":"/usr/java/jdk1.8.0_144/jre","java.version":"1.8.0_144","sun.io.unicode.encoding":"UnicodeLittle"},"Classpath Entries":{"/opt/cloudera/parcels/CDH/jars/jackson-mapper-asl-1.8.8.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/joni-2.1.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jackson-core-2.2.3.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-external-blockcache-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/metrics-json-3.1.5.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/xz-1.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jackson-annotations-2.2.3.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/cglib-2.2.1-v20090111.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-procedure-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/parquet-jackson-1.10.1.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/asm-3.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/xml-apis-1.3.04.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/metrics-jvm-3.1.5.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-dbcp-1.4.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-streaming_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-xml-9.4.12.v20180830.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/minlog-1.3.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hsqldb-1.8.0.10.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/datanucleus-rdbms-3.2.9.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/pmml-model-1.4.8.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spire-macros_2.12-0.13.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/zstd-jni-1.3.2-2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/httpcore-4.2.5.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jta-1.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-yarn_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-logging-1.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/activation-1.1.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/xbean-asm7-shaded-4.12.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/kafka-0.9/paranamer-2.8.jar":"System Classpath","/opt/cloudera/parcels/CDH/lib/hadoop/NOTICE.txt":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/arrow-format-0.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/httpclient-4.2.5.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/netty-3.9.9.Final.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/htrace-core-3.1.0-incubating.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/javax.ws.rs-api-2.0.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jersey-container-servlet-core-2.22.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-mapreduce-client-nativetask-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jersey-client-2.22.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jackson-xc-1.8.8.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-configuration-1.6.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/commons-math3-3.1.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jsp-api-2.1.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-auth-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-yarn-registry-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/JavaEWAH-0.3.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/metrics-graphite-3.1.5.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/joda-time-2.9.9.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/aopalliance-1.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-gridmix-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/commons-compress-1.4.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/javolution-5.5.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-beanutils-1.7.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/flatbuffers-java-1.9.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/core-1.1.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/antlr-runtime-3.4.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-hdfs-nfs-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-hdfs-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-net-3.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/curator-framework-2.7.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/snappy-java-1.1.7.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-assembly_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-examples-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/shapeless_2.12-2.3.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-mapreduce-client-hs-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/xercesImpl-2.9.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jersey-server-2.22.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-lang-2.6.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jtransforms-2.4.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/snappy-0.2.jar":"System Classpath","/etc/spark2/conf/yarn-conf/":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-cli-1.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jackson-core-2.9.8.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/javax.annotation-api-1.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-util-6.1.26.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/stax-api-1.0-2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/javassist-3.18.1-GA.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-yarn-server-applicationhistoryservice-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/kafka-0.9/kafka-clients-0.9.0-kafka-2.0.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/guice-3.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/antlr4-runtime-4.7.1.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/core-3.1.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/opencsv-2.3.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-datajoin-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jersey-common-2.22.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jdo-api-3.0.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-webapp-9.4.12.v20180830.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-compiler-3.0.11.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-annotations-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/libthrift-0.9.3.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/kafka-0.9/kafka_2.11-0.9.0-kafka-2.0.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-azure-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-proxy-9.4.12.v20180830.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/antlr-2.7.7.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jettison-1.1.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jackson-core-asl-1.8.8.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/libfb303-0.9.3.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/zookeeper-3.4.5-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-resource-bundle-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jaxb-api-2.2.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-streaming-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-hadoop2-compat-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/pyrolite-4.13.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/activation-1.1.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-mapreduce-client-common-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/aircompressor-0.10.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-ant-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jline-2.11.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/netty-all-4.0.23.Final.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jersey-media-jaxb-2.22.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-servlets-9.4.12.v20180830.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/scala-xml_2.12-1.0.6.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-graphx_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-mapreduce-client-hs-plugins-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-yarn-server-web-proxy-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/commons-math-2.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/avro-1.8.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-kvstore_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/objenesis-2.5.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/stream-2.7.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-aws-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jackson-module-scala_2.12-2.9.8.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/spymemcached-2.11.6.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/api-util-1.0.0-M20.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/parquet-hadoop-1.10.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/orc-core-1.5.5-nohive.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/mockito-all-1.8.5.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jets3t-0.9.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jackson-xc-1.9.13.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-rsgroup-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/hive-metastore-1.2.1.spark2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/py4j-0.10.8.1.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-yarn-applications-distributedshell-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/osgi-resource-locator-1.0.1.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-rumen-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jsch-0.1.42.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/kafka-0.9/zkclient-0.7.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/xmlenc-0.52.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-io-2.4.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/metrics-core-3.1.5.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jamon-runtime-2.4.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/javax.inject-2.4.0-b34.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-common-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/parquet-common-1.10.1.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/netty-3.10.5.Final.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-mapreduce-client-app-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/machinist_2.12-0.6.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-util-9.4.12.v20180830.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-network-shuffle_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/chill_2.12-0.9.3.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/findbugs-annotations-1.3.9-1.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-rest-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-hadoop-compat-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/ST4-4.0.4.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jruby-cloudera-1.0.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-hive_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/leveldbjni-all-1.8.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-core_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-distcp-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-server-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/kafka-0.9/metrics-core-2.2.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/htrace-core-3.2.0-incubating.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/orc-mapreduce-1.5.5-nohive.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jcl-over-slf4j-1.7.16.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-digester-1.8.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-http-9.4.12.v20180830.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/json4s-jackson_2.12-3.5.3.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/metrics-core-2.2.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-math3-3.4.1.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/slf4j-log4j12-1.7.5.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-mapreduce-client-jobclient-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-shell-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-server-9.4.12.v20180830.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/chill-java-0.9.3.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/hk2-locator-2.4.0-b34.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/stax-api-1.0.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/scala-parser-combinators_2.12-1.1.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/RoaringBitmap-0.5.11.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jsp-api-2.1-6.1.14.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/log4j-1.2.16.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/javax.inject-1.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jasper-compiler-5.5.23.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-security-9.4.12.v20180830.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-mapreduce-client-shuffle-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-nfs-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/netty-all-4.1.30.Final.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jaxb-api-2.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/breeze-macros_2.12-0.13.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jodd-core-3.5.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/azure-data-lake-store-sdk-2.2.9.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/guava-12.0.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-codec-1.10.jar":"System Classpath","/usr/java/jdk1.8.0_144/lib/tools.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-yarn-server-nodemanager-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-sql_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/hive-exec-1.2.1.spark2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jcodings-1.0.8.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-azure-datalake-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/guava-11.0.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/avro-mapred-1.8.2-hadoop2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-catalyst_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-io-9.4.12.v20180830.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-yarn-applications-unmanaged-am-launcher-2.6.0-cdh5.15.2.jar":"System Classpath","/etc/spark2/conf/":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-sslengine-6.1.26.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/httpclient-4.5.6.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-continuation-9.4.12.v20180830.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/json4s-ast_2.12-3.5.3.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-jndi-9.4.12.v20180830.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jsr305-3.0.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/avro-1.7.6-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/univocity-parsers-2.7.3.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-annotations-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/api-asn1-api-1.0.0-M20.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jettison-1.3.3.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/hk2-utils-2.4.0-b34.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/hppc-0.7.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/logredactor-1.0.3.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/arrow-vector-0.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jaxb-impl-2.2.3-1.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-archives-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-plus-9.4.12.v20180830.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-client-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hue-plugins-3.9.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/high-scale-lib-1.1.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jackson-jaxrs-1.9.13.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-collections-3.2.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/scala-library-2.12.8.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/java-xmlbuilder-0.4.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/apache-log4j-extras-1.2.17.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/parquet-format-2.4.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/paranamer-2.8.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spire_2.12-0.13.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/disruptor-3.3.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-mllib-local_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-yarn-common-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-pool-1.5.4.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-servlet-9.4.12.v20180830.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/arpack_combined_all-0.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jackson-annotations-2.9.8.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jsp-2.1-6.1.14.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/log4j-1.2.17.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-repl_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/derby-10.12.1.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/orc-shims-1.5.5.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-mapreduce-client-core-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/aopalliance-repackaged-2.4.0-b34.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/hk2-api-2.4.0-b34.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/commons-daemon-1.0.13.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/ivy-2.4.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hamcrest-core-1.3.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-lang3-3.8.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/kafka-0.9/spark-streaming-kafka-0-8_2.11-2.4.0.cloudera1-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jackson-databind-2.2.3.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/kryo-shaded-4.0.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/scala-reflect-2.12.8.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-mapreduce-examples-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/httpcore-4.4.10.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/arrow-memory-0.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/lib/hadoop/LICENSE.txt":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-compress-1.8.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/curator-client-2.7.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/oro-2.0.8.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-thrift-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/avro-ipc-1.8.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-httpclient-3.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/parquet-encoding-1.10.1.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/okhttp-2.4.0.jar":"System Classpath","spark://test-1.vpc.company.com:34194/jars/spark3-tests-0.1.0-cdh5.9.0-SNAPSHOT-jar-with-dependencies.jar":"Added By User","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/scala-compiler-2.12.8.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-prefix-tree-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-6.1.26.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-yarn-api-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/commons-codec-1.9.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-beanutils-core-1.8.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/kafka-0.9/lz4-1.3.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/janino-3.0.11.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/libthrift-0.12.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-launcher_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/json4s-core_2.12-3.5.3.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/lz4-java-1.5.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jackson-jaxrs-1.8.8.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-network-common_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/macro-compat_2.12-1.1.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/breeze_2.12-0.13.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-unsafe_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-sls-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/snappy-java-1.0.4.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jersey-guava-2.22.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/validation-api-1.1.0.Final.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/microsoft-windowsazure-storage-sdk-0.6.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/compress-lzf-1.0.3.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-openstack-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-mllib_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jetty-client-9.4.12.v20180830.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-it-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-extras-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jersey-container-servlet-2.22.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/javax.servlet-api-3.1.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/curator-recipes-2.7.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/parquet-column-1.10.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/unused-1.0.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/commons-crypto-1.0.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jackson-databind-2.9.8.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/commons-codec-1.4.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/htrace-core4-4.0.1-incubating.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/guava-14.0.1.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/aws-java-sdk-bundle-1.11.134.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/zookeeper-3.4.6.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jackson-module-paranamer-2.9.8.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hbase-protocol-1.2.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/jasper-runtime-5.5.23.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/json4s-scalap_2.12-3.5.3.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/commons-logging-1.1.3.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-yarn-server-resourcemanager-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-archive-logs-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/paranamer-2.3.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/metrics-core-3.0.2.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/commons-beanutils-1.9.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jul-to-slf4j-1.7.16.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/stringtemplate-3.2.1.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/gson-2.2.4.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/okio-1.4.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/slf4j-api-1.7.5.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/jsr305-3.0.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/bonecp-0.8.0.RELEASE.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-sketch_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/commons-el-1.0.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/spark-tags_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/datanucleus-core-3.2.10.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/protobuf-java-2.5.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-yarn-server-common-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/parquet-hadoop-bundle-1.6.0.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-common-2.6.0-cdh5.15.2.jar":"System Classpath","/opt/cloudera/parcels/SPARK2/lib/spark2/jars/xz-1.5.jar":"System Classpath","/opt/cloudera/parcels/CDH/jars/hadoop-yarn-client-2.6.0-cdh5.15.2.jar":"System Classpath"}} {"Event":"SparkListenerApplicationStart","App Name":"LargeBlocks","App ID":"application_1553914137147_0018","Timestamp":1554755984286,"User":"systest"} {"Event":"SparkListenerExecutorAdded","Timestamp":1554755994596,"Executor ID":"1","Executor Info":{"Host":"test-2.vpc.company.com","Total Cores":1,"Log Urls":{"stdout":"http://test-2.vpc.company.com:8042/node/containerlogs/container_1553914137147_0018_01_000002/systest/stdout?start=-4096","stderr":"http://test-2.vpc.company.com:8042/node/containerlogs/container_1553914137147_0018_01_000002/systest/stderr?start=-4096"},"Attributes":{"NM_HTTP_ADDRESS":"test-2.vpc.company.com:8042","USER":"systest","LOG_FILES":"stderr,stdout","NM_HTTP_PORT":"8042","CLUSTER_ID":"","NM_PORT":"8041","HTTP_SCHEME":"http://","NM_HOST":"test-2.vpc.company.com","CONTAINER_ID":"container_1553914137147_0018_01_000002"}}} {"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"1","Host":"test-2.vpc.company.com","Port":43764},"Maximum Memory":3820172083,"Timestamp":1554755994649,"Maximum Onheap Memory":3820172083,"Maximum Offheap Memory":0} diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 3f309819065be..4d157b9607000 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -174,7 +174,7 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex private def testCaching(conf: SparkConf, storageLevel: StorageLevel): Unit = { sc = new SparkContext(conf.setMaster(clusterUrl).setAppName("test")) - TestUtils.waitUntilExecutorsUp(sc, 2, 30000) + TestUtils.waitUntilExecutorsUp(sc, 2, 60000) val data = sc.parallelize(1 to 1000, 10) val cachedData = data.persist(storageLevel) assert(cachedData.count === 1000) diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index 8d958494d52be..8fa33f4915ea4 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -29,6 +29,7 @@ import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.internal.config import org.apache.spark.internal.config.Tests.TEST_SCHEDULE_INTERVAL import org.apache.spark.metrics.MetricsSystem +import org.apache.spark.resource.{ExecutorResourceRequests, ResourceProfile, ResourceProfileBuilder, ResourceProfileManager, TaskResourceRequests} import org.apache.spark.resource.ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo @@ -45,6 +46,9 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { private val managers = new mutable.ListBuffer[ExecutorAllocationManager]() private var listenerBus: LiveListenerBus = _ private var client: ExecutorAllocationClient = _ + private val clock = new SystemClock() + private var rpManager: ResourceProfileManager = _ + override def beforeEach(): Unit = { super.beforeEach() @@ -108,65 +112,257 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { test("starting state") { val manager = createManager(createConf()) - assert(numExecutorsTarget(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) assert(executorsPendingToRemove(manager).isEmpty) assert(addTime(manager) === ExecutorAllocationManager.NOT_SET) } - test("add executors") { + test("add executors default profile") { val manager = createManager(createConf(1, 10, 1)) post(SparkListenerStageSubmitted(createStageInfo(0, 1000))) + val updatesNeeded = + new mutable.HashMap[ResourceProfile, ExecutorAllocationManager.TargetNumUpdates] + + // Keep adding until the limit is reached + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 2) + assert(numExecutorsToAddForDefaultProfile(manager) === 2) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 4) + assert(numExecutorsToAddForDefaultProfile(manager) === 4) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 4) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 8) + assert(numExecutorsToAddForDefaultProfile(manager) === 8) + // reached the limit of 10 + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 0) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) + + // Register previously requested executors + onExecutorAddedDefaultProfile(manager, "first") + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + onExecutorAddedDefaultProfile(manager, "second") + onExecutorAddedDefaultProfile(manager, "third") + onExecutorAddedDefaultProfile(manager, "fourth") + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + onExecutorAddedDefaultProfile(manager, "first") // duplicates should not count + onExecutorAddedDefaultProfile(manager, "second") + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + + // Try adding again + // This should still fail because the number pending + running is still at the limit + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 0) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 0) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) + } + + test("add executors multiple profiles") { + val manager = createManager(createConf(1, 10, 1)) + post(SparkListenerStageSubmitted(createStageInfo(0, 1000, rp = defaultProfile))) + val rp1 = new ResourceProfileBuilder() + val execReqs = new ExecutorResourceRequests().cores(4).resource("gpu", 4) + val taskReqs = new TaskResourceRequests().cpus(1).resource("gpu", 1) + rp1.require(execReqs).require(taskReqs) + val rprof1 = rp1.build + rpManager.addResourceProfile(rprof1) + post(SparkListenerStageSubmitted(createStageInfo(1, 1000, rp = rprof1))) + val updatesNeeded = + new mutable.HashMap[ResourceProfile, ExecutorAllocationManager.TargetNumUpdates] + // Keep adding until the limit is reached - assert(numExecutorsTarget(manager) === 1) - assert(numExecutorsToAdd(manager) === 1) - assert(addExecutors(manager) === 1) - assert(numExecutorsTarget(manager) === 2) - assert(numExecutorsToAdd(manager) === 2) - assert(addExecutors(manager) === 2) - assert(numExecutorsTarget(manager) === 4) - assert(numExecutorsToAdd(manager) === 4) - assert(addExecutors(manager) === 4) - assert(numExecutorsTarget(manager) === 8) - assert(numExecutorsToAdd(manager) === 8) - assert(addExecutors(manager) === 2) // reached the limit of 10 - assert(numExecutorsTarget(manager) === 10) - assert(numExecutorsToAdd(manager) === 1) - assert(addExecutors(manager) === 0) - assert(numExecutorsTarget(manager) === 10) - assert(numExecutorsToAdd(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + assert(numExecutorsToAdd(manager, rprof1) === 1) + assert(numExecutorsTarget(manager, rprof1.id) === 1) + assert(addExecutorsToTarget(manager, updatesNeeded, rprof1) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 2) + assert(numExecutorsToAddForDefaultProfile(manager) === 2) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + assert(numExecutorsToAdd(manager, rprof1) === 2) + assert(numExecutorsTarget(manager, rprof1.id) === 2) + assert(addExecutorsToTarget(manager, updatesNeeded, rprof1) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 4) + assert(numExecutorsToAddForDefaultProfile(manager) === 4) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 4) + assert(numExecutorsToAdd(manager, rprof1) === 4) + assert(numExecutorsTarget(manager, rprof1.id) === 4) + assert(addExecutorsToTarget(manager, updatesNeeded, rprof1) === 4) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 8) + assert(numExecutorsToAddForDefaultProfile(manager) === 8) + // reached the limit of 10 + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + assert(numExecutorsToAdd(manager, rprof1) === 8) + assert(numExecutorsTarget(manager, rprof1.id) === 8) + assert(addExecutorsToTarget(manager, updatesNeeded, rprof1) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 0) + assert(numExecutorsToAdd(manager, rprof1) === 1) + assert(numExecutorsTarget(manager, rprof1.id) === 10) + assert(addExecutorsToTarget(manager, updatesNeeded, rprof1) === 0) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) + assert(numExecutorsToAdd(manager, rprof1) === 1) + assert(numExecutorsTarget(manager, rprof1.id) === 10) // Register previously requested executors - onExecutorAdded(manager, "first") - assert(numExecutorsTarget(manager) === 10) - onExecutorAdded(manager, "second") - onExecutorAdded(manager, "third") - onExecutorAdded(manager, "fourth") - assert(numExecutorsTarget(manager) === 10) - onExecutorAdded(manager, "first") // duplicates should not count - onExecutorAdded(manager, "second") - assert(numExecutorsTarget(manager) === 10) + onExecutorAddedDefaultProfile(manager, "first") + onExecutorAdded(manager, "firstrp1", rprof1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + assert(numExecutorsTarget(manager, rprof1.id) === 10) + onExecutorAddedDefaultProfile(manager, "second") + onExecutorAddedDefaultProfile(manager, "third") + onExecutorAddedDefaultProfile(manager, "fourth") + onExecutorAdded(manager, "secondrp1", rprof1) + onExecutorAdded(manager, "thirdrp1", rprof1) + onExecutorAdded(manager, "fourthrp1", rprof1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + assert(numExecutorsTarget(manager, rprof1.id) === 10) + onExecutorAddedDefaultProfile(manager, "first") // duplicates should not count + onExecutorAddedDefaultProfile(manager, "second") + onExecutorAdded(manager, "firstrp1", rprof1) + onExecutorAdded(manager, "secondrp1", rprof1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + assert(numExecutorsTarget(manager, rprof1.id) === 10) // Try adding again // This should still fail because the number pending + running is still at the limit - assert(addExecutors(manager) === 0) - assert(numExecutorsTarget(manager) === 10) - assert(numExecutorsToAdd(manager) === 1) - assert(addExecutors(manager) === 0) - assert(numExecutorsTarget(manager) === 10) - assert(numExecutorsToAdd(manager) === 1) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 0) + assert(addExecutorsToTarget(manager, updatesNeeded, rprof1) === 0) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) + assert(numExecutorsToAdd(manager, rprof1) === 1) + assert(numExecutorsTarget(manager, rprof1.id) === 10) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 0) + assert(addExecutorsToTarget(manager, updatesNeeded, rprof1) === 0) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) + assert(numExecutorsToAdd(manager, rprof1) === 1) + assert(numExecutorsTarget(manager, rprof1.id) === 10) + } + + test("remove executors multiple profiles") { + val manager = createManager(createConf(5, 10, 5)) + val rp1 = new ResourceProfileBuilder() + val execReqs = new ExecutorResourceRequests().cores(4).resource("gpu", 4) + val taskReqs = new TaskResourceRequests().cpus(1).resource("gpu", 1) + rp1.require(execReqs).require(taskReqs) + val rprof1 = rp1.build + val rp2 = new ResourceProfileBuilder() + val execReqs2 = new ExecutorResourceRequests().cores(1) + val taskReqs2 = new TaskResourceRequests().cpus(1) + rp2.require(execReqs2).require(taskReqs2) + val rprof2 = rp2.build + rpManager.addResourceProfile(rprof1) + rpManager.addResourceProfile(rprof2) + post(SparkListenerStageSubmitted(createStageInfo(1, 10, rp = rprof1))) + post(SparkListenerStageSubmitted(createStageInfo(2, 10, rp = rprof2))) + + (1 to 10).map(_.toString).foreach { id => onExecutorAdded(manager, id, rprof1) } + (11 to 20).map(_.toString).foreach { id => onExecutorAdded(manager, id, rprof2) } + (21 to 30).map(_.toString).foreach { id => onExecutorAdded(manager, id, defaultProfile) } + + // Keep removing until the limit is reached + assert(executorsPendingToRemove(manager).isEmpty) + assert(removeExecutor(manager, "1", rprof1.id)) + assert(executorsPendingToRemove(manager).size === 1) + assert(executorsPendingToRemove(manager).contains("1")) + assert(removeExecutor(manager, "11", rprof2.id)) + assert(removeExecutor(manager, "2", rprof1.id)) + assert(executorsPendingToRemove(manager).size === 3) + assert(executorsPendingToRemove(manager).contains("2")) + assert(executorsPendingToRemove(manager).contains("11")) + assert(removeExecutor(manager, "21", defaultProfile.id)) + assert(removeExecutor(manager, "3", rprof1.id)) + assert(removeExecutor(manager, "4", rprof1.id)) + assert(executorsPendingToRemove(manager).size === 6) + assert(executorsPendingToRemove(manager).contains("21")) + assert(executorsPendingToRemove(manager).contains("3")) + assert(executorsPendingToRemove(manager).contains("4")) + assert(removeExecutor(manager, "5", rprof1.id)) + assert(!removeExecutor(manager, "6", rprof1.id)) // reached the limit of 5 + assert(executorsPendingToRemove(manager).size === 7) + assert(executorsPendingToRemove(manager).contains("5")) + assert(!executorsPendingToRemove(manager).contains("6")) + + // Kill executors previously requested to remove + onExecutorRemoved(manager, "1") + assert(executorsPendingToRemove(manager).size === 6) + assert(!executorsPendingToRemove(manager).contains("1")) + onExecutorRemoved(manager, "2") + onExecutorRemoved(manager, "3") + assert(executorsPendingToRemove(manager).size === 4) + assert(!executorsPendingToRemove(manager).contains("2")) + assert(!executorsPendingToRemove(manager).contains("3")) + onExecutorRemoved(manager, "2") // duplicates should not count + onExecutorRemoved(manager, "3") + assert(executorsPendingToRemove(manager).size === 4) + onExecutorRemoved(manager, "4") + onExecutorRemoved(manager, "5") + assert(executorsPendingToRemove(manager).size === 2) + assert(executorsPendingToRemove(manager).contains("11")) + assert(executorsPendingToRemove(manager).contains("21")) + + // Try removing again + // This should still fail because the number pending + running is still at the limit + assert(!removeExecutor(manager, "7", rprof1.id)) + assert(executorsPendingToRemove(manager).size === 2) + assert(!removeExecutor(manager, "8", rprof1.id)) + assert(executorsPendingToRemove(manager).size === 2) + + // make sure rprof2 has the same min limit or 5 + assert(removeExecutor(manager, "12", rprof2.id)) + assert(removeExecutor(manager, "13", rprof2.id)) + assert(removeExecutor(manager, "14", rprof2.id)) + assert(removeExecutor(manager, "15", rprof2.id)) + assert(!removeExecutor(manager, "16", rprof2.id)) // reached the limit of 5 + assert(executorsPendingToRemove(manager).size === 6) + assert(!executorsPendingToRemove(manager).contains("16")) + onExecutorRemoved(manager, "11") + onExecutorRemoved(manager, "12") + onExecutorRemoved(manager, "13") + onExecutorRemoved(manager, "14") + onExecutorRemoved(manager, "15") + assert(executorsPendingToRemove(manager).size === 1) } def testAllocationRatio(cores: Int, divisor: Double, expected: Int): Unit = { + val updatesNeeded = + new mutable.HashMap[ResourceProfile, ExecutorAllocationManager.TargetNumUpdates] val conf = createConf(3, 15) .set(config.DYN_ALLOCATION_EXECUTOR_ALLOCATION_RATIO, divisor) .set(config.EXECUTOR_CORES, cores) val manager = createManager(conf) post(SparkListenerStageSubmitted(createStageInfo(0, 20))) for (i <- 0 to 5) { - addExecutors(manager) + addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) } - assert(numExecutorsTarget(manager) === expected) + assert(numExecutorsTargetForDefaultProfileId(manager) === expected) } test("executionAllocationRatio is correctly handled") { @@ -185,127 +381,158 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { val manager = createManager(createConf(0, 10, 0)) post(SparkListenerStageSubmitted(createStageInfo(0, 5))) + val updatesNeeded = + new mutable.HashMap[ResourceProfile, ExecutorAllocationManager.TargetNumUpdates] + // Verify that we're capped at number of tasks in the stage - assert(numExecutorsTarget(manager) === 0) - assert(numExecutorsToAdd(manager) === 1) - assert(addExecutors(manager) === 1) - assert(numExecutorsTarget(manager) === 1) - assert(numExecutorsToAdd(manager) === 2) - assert(addExecutors(manager) === 2) - assert(numExecutorsTarget(manager) === 3) - assert(numExecutorsToAdd(manager) === 4) - assert(addExecutors(manager) === 2) - assert(numExecutorsTarget(manager) === 5) - assert(numExecutorsToAdd(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 0) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) + assert(numExecutorsToAddForDefaultProfile(manager) === 2) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 3) + assert(numExecutorsToAddForDefaultProfile(manager) === 4) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 5) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) // Verify that running a task doesn't affect the target post(SparkListenerStageSubmitted(createStageInfo(1, 3))) post(SparkListenerExecutorAdded( 0L, "executor-1", new ExecutorInfo("host1", 1, Map.empty, Map.empty))) post(SparkListenerTaskStart(1, 0, createTaskInfo(0, 0, "executor-1"))) - assert(numExecutorsTarget(manager) === 5) - assert(addExecutors(manager) === 1) - assert(numExecutorsTarget(manager) === 6) - assert(numExecutorsToAdd(manager) === 2) - assert(addExecutors(manager) === 2) - assert(numExecutorsTarget(manager) === 8) - assert(numExecutorsToAdd(manager) === 4) - assert(addExecutors(manager) === 0) - assert(numExecutorsTarget(manager) === 8) - assert(numExecutorsToAdd(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 5) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 6) + assert(numExecutorsToAddForDefaultProfile(manager) === 2) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 8) + assert(numExecutorsToAddForDefaultProfile(manager) === 4) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 0) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 8) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) // Verify that re-running a task doesn't blow things up post(SparkListenerStageSubmitted(createStageInfo(2, 3))) post(SparkListenerTaskStart(2, 0, createTaskInfo(0, 0, "executor-1"))) post(SparkListenerTaskStart(2, 0, createTaskInfo(1, 0, "executor-1"))) - assert(addExecutors(manager) === 1) - assert(numExecutorsTarget(manager) === 9) - assert(numExecutorsToAdd(manager) === 2) - assert(addExecutors(manager) === 1) - assert(numExecutorsTarget(manager) === 10) - assert(numExecutorsToAdd(manager) === 1) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 9) + assert(numExecutorsToAddForDefaultProfile(manager) === 2) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) // Verify that running a task once we're at our limit doesn't blow things up post(SparkListenerTaskStart(2, 0, createTaskInfo(0, 1, "executor-1"))) - assert(addExecutors(manager) === 0) - assert(numExecutorsTarget(manager) === 10) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 0) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 10) } test("add executors when speculative tasks added") { val manager = createManager(createConf(0, 10, 0)) + val updatesNeeded = + new mutable.HashMap[ResourceProfile, ExecutorAllocationManager.TargetNumUpdates] + + post(SparkListenerStageSubmitted(createStageInfo(1, 2))) // Verify that we're capped at number of tasks including the speculative ones in the stage post(SparkListenerSpeculativeTaskSubmitted(1)) - assert(numExecutorsTarget(manager) === 0) - assert(numExecutorsToAdd(manager) === 1) - assert(addExecutors(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 0) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) post(SparkListenerSpeculativeTaskSubmitted(1)) post(SparkListenerSpeculativeTaskSubmitted(1)) - post(SparkListenerStageSubmitted(createStageInfo(1, 2))) - assert(numExecutorsTarget(manager) === 1) - assert(numExecutorsToAdd(manager) === 2) - assert(addExecutors(manager) === 2) - assert(numExecutorsTarget(manager) === 3) - assert(numExecutorsToAdd(manager) === 4) - assert(addExecutors(manager) === 2) - assert(numExecutorsTarget(manager) === 5) - assert(numExecutorsToAdd(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) + assert(numExecutorsToAddForDefaultProfile(manager) === 2) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 3) + assert(numExecutorsToAddForDefaultProfile(manager) === 4) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 5) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) // Verify that running a task doesn't affect the target post(SparkListenerTaskStart(1, 0, createTaskInfo(0, 0, "executor-1"))) - assert(numExecutorsTarget(manager) === 5) - assert(addExecutors(manager) === 0) - assert(numExecutorsToAdd(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 5) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 0) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) // Verify that running a speculative task doesn't affect the target post(SparkListenerTaskStart(1, 0, createTaskInfo(1, 0, "executor-2", true))) - assert(numExecutorsTarget(manager) === 5) - assert(addExecutors(manager) === 0) - assert(numExecutorsToAdd(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 5) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 0) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) } test("SPARK-30511 remove executors when speculative tasks end") { val clock = new ManualClock() val stage = createStageInfo(0, 40) - val manager = createManager(createConf(0, 10, 0).set(config.EXECUTOR_CORES, 4), clock = clock) + val conf = createConf(0, 10, 0).set(config.EXECUTOR_CORES, 4) + val manager = createManager(conf, clock = clock) + val updatesNeeded = + new mutable.HashMap[ResourceProfile, ExecutorAllocationManager.TargetNumUpdates] post(SparkListenerStageSubmitted(stage)) - assert(addExecutors(manager) === 1) - assert(addExecutors(manager) === 2) - assert(addExecutors(manager) === 4) - assert(addExecutors(manager) === 3) - - (0 to 9).foreach(execId => onExecutorAdded(manager, execId.toString)) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 4) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 3) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + + (0 to 9).foreach(execId => onExecutorAddedDefaultProfile(manager, execId.toString)) (0 to 39).map { i => createTaskInfo(i, i, executorId = s"${i / 4}")}.foreach { info => post(SparkListenerTaskStart(0, 0, info)) } - assert(numExecutorsTarget(manager) === 10) - assert(maxNumExecutorsNeeded(manager) == 10) + assert(numExecutorsTarget(manager, defaultProfile.id) === 10) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 10) // 30 tasks (0 - 29) finished (0 to 29).map { i => createTaskInfo(i, i, executorId = s"${i / 4}")}.foreach { info => post(SparkListenerTaskEnd(0, 0, null, Success, info, new ExecutorMetrics, null)) } clock.advance(1000) manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) - assert(numExecutorsTarget(manager) === 3) - assert(maxNumExecutorsNeeded(manager) == 3) - (0 to 6).foreach { i => assert(removeExecutor(manager, i.toString))} + assert(numExecutorsTarget(manager, defaultProfile.id) === 3) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 3) + (0 to 6).foreach { i => assert(removeExecutorDefaultProfile(manager, i.toString))} (0 to 6).foreach { i => onExecutorRemoved(manager, i.toString)} // 10 speculative tasks (30 - 39) launch for the remaining tasks (30 to 39).foreach { _ => post(SparkListenerSpeculativeTaskSubmitted(0))} - assert(addExecutors(manager) === 1) - assert(addExecutors(manager) === 1) - assert(numExecutorsTarget(manager) == 5) - assert(maxNumExecutorsNeeded(manager) == 5) - (10 to 12).foreach(execId => onExecutorAdded(manager, execId.toString)) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTarget(manager, defaultProfile.id) == 5) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 5) + (10 to 12).foreach(execId => onExecutorAddedDefaultProfile(manager, execId.toString)) (40 to 49).map { i => createTaskInfo(taskId = i, taskIndex = i - 10, executorId = s"${i / 4}", speculative = true)} .foreach { info => post(SparkListenerTaskStart(0, 0, info))} clock.advance(1000) manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) - assert(numExecutorsTarget(manager) == 5) // At this point, we still have 6 executors running - assert(maxNumExecutorsNeeded(manager) == 5) + // At this point, we still have 6 executors running + assert(numExecutorsTarget(manager, defaultProfile.id) == 5) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 5) // 6 speculative tasks (40 - 45) finish before the original tasks, with 4 speculative remaining (40 to 45).map { i => @@ -314,9 +541,9 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { info => post(SparkListenerTaskEnd(0, 0, null, Success, info, new ExecutorMetrics, null))} clock.advance(1000) manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) - assert(numExecutorsTarget(manager) === 4) - assert(maxNumExecutorsNeeded(manager) == 4) - assert(removeExecutor(manager, "10")) + assert(numExecutorsTarget(manager, defaultProfile.id) === 4) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 4) + assert(removeExecutorDefaultProfile(manager, "10")) onExecutorRemoved(manager, "10") // At this point, we still have 5 executors running: ["7", "8", "9", "11", "12"] @@ -327,9 +554,9 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { SparkListenerTaskEnd(0, 0, null, TaskKilled("test"), info, new ExecutorMetrics, null))} clock.advance(1000) manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) - assert(numExecutorsTarget(manager) === 2) - assert(maxNumExecutorsNeeded(manager) == 2) - (7 to 8).foreach { i => assert(removeExecutor(manager, i.toString))} + assert(numExecutorsTarget(manager, defaultProfile.id) === 2) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 2) + (7 to 8).foreach { i => assert(removeExecutorDefaultProfile(manager, i.toString))} (7 to 8).foreach { i => onExecutorRemoved(manager, i.toString)} // At this point, we still have 3 executors running: ["9", "11", "12"] @@ -343,8 +570,8 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { // tasks running. Target lowers to 2, but still hold 3 executors ["9", "11", "12"] clock.advance(1000) manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) - assert(numExecutorsTarget(manager) === 2) - assert(maxNumExecutorsNeeded(manager) == 2) + assert(numExecutorsTarget(manager, defaultProfile.id) === 2) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 2) // At this point, we still have 3 executors running: ["9", "11", "12"] // Task 37 and 47 succeed at the same time @@ -357,9 +584,9 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { // tasks running clock.advance(1000) manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) - assert(numExecutorsTarget(manager) === 1) - assert(maxNumExecutorsNeeded(manager) == 1) - assert(removeExecutor(manager, "11")) + assert(numExecutorsTarget(manager, defaultProfile.id) === 1) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 1) + assert(removeExecutorDefaultProfile(manager, "11")) onExecutorRemoved(manager, "11") // At this point, we still have 2 executors running: ["9", "12"] @@ -372,14 +599,14 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { clock.advance(1000) manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) // maxNeeded = 1, allocate one more to satisfy speculation locality requirement - assert(numExecutorsTarget(manager) === 2) - assert(maxNumExecutorsNeeded(manager) == 2) + assert(numExecutorsTarget(manager, defaultProfile.id) === 2) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 2) post(SparkListenerTaskStart(0, 0, createTaskInfo(50, 39, executorId = "12", speculative = true))) clock.advance(1000) manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) - assert(numExecutorsTarget(manager) === 1) - assert(maxNumExecutorsNeeded(manager) == 1) + assert(numExecutorsTarget(manager, defaultProfile.id) === 1) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 1) // Task 39 and 48 succeed, task 50 killed post(SparkListenerTaskEnd(0, 0, null, Success, @@ -391,11 +618,11 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { post(SparkListenerStageCompleted(stage)) clock.advance(1000) manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) - assert(numExecutorsTarget(manager) === 0) - assert(maxNumExecutorsNeeded(manager) == 0) - assert(removeExecutor(manager, "9")) + assert(numExecutorsTarget(manager, defaultProfile.id) === 0) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 0) + assert(removeExecutorDefaultProfile(manager, "9")) onExecutorRemoved(manager, "9") - assert(removeExecutor(manager, "12")) + assert(removeExecutorDefaultProfile(manager, "12")) onExecutorRemoved(manager, "12") } @@ -417,43 +644,49 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { post(SparkListenerStageCompleted(stage)) // There are still two tasks that belong to the zombie stage running. - assert(totalRunningTasks(manager) === 2) + assert(totalRunningTasksPerResourceProfile(manager) === 2) // submit another attempt for the stage. We count completions from the first zombie attempt val stageAttempt1 = createStageInfo(stage.stageId, 5, attemptId = 1) post(SparkListenerStageSubmitted(stageAttempt1)) post(SparkListenerTaskEnd(0, 0, null, Success, taskInfo1, new ExecutorMetrics, null)) - assert(totalRunningTasks(manager) === 1) + assert(totalRunningTasksPerResourceProfile(manager) === 1) val attemptTaskInfo1 = createTaskInfo(3, 0, "executor-1") val attemptTaskInfo2 = createTaskInfo(4, 1, "executor-1") post(SparkListenerTaskStart(0, 1, attemptTaskInfo1)) post(SparkListenerTaskStart(0, 1, attemptTaskInfo2)) - assert(totalRunningTasks(manager) === 3) + assert(totalRunningTasksPerResourceProfile(manager) === 3) post(SparkListenerTaskEnd(0, 1, null, Success, attemptTaskInfo1, new ExecutorMetrics, null)) - assert(totalRunningTasks(manager) === 2) + assert(totalRunningTasksPerResourceProfile(manager) === 2) post(SparkListenerTaskEnd(0, 0, null, Success, taskInfo2, new ExecutorMetrics, null)) - assert(totalRunningTasks(manager) === 1) + assert(totalRunningTasksPerResourceProfile(manager) === 1) post(SparkListenerTaskEnd(0, 1, null, Success, attemptTaskInfo2, new ExecutorMetrics, null)) - assert(totalRunningTasks(manager) === 0) + assert(totalRunningTasksPerResourceProfile(manager) === 0) } testRetry("cancel pending executors when no longer needed") { val manager = createManager(createConf(0, 10, 0)) post(SparkListenerStageSubmitted(createStageInfo(2, 5))) - assert(numExecutorsTarget(manager) === 0) - assert(numExecutorsToAdd(manager) === 1) - assert(addExecutors(manager) === 1) - assert(numExecutorsTarget(manager) === 1) - assert(numExecutorsToAdd(manager) === 2) - assert(addExecutors(manager) === 2) - assert(numExecutorsTarget(manager) === 3) + val updatesNeeded = + new mutable.HashMap[ResourceProfile, ExecutorAllocationManager.TargetNumUpdates] + + assert(numExecutorsTargetForDefaultProfileId(manager) === 0) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) + assert(numExecutorsToAddForDefaultProfile(manager) === 2) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 3) val task1Info = createTaskInfo(0, 0, "executor-1") post(SparkListenerTaskStart(2, 0, task1Info)) - assert(numExecutorsToAdd(manager) === 4) - assert(addExecutors(manager) === 2) + assert(numExecutorsToAddForDefaultProfile(manager) === 4) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) val task2Info = createTaskInfo(1, 0, "executor-1") post(SparkListenerTaskStart(2, 0, task2Info)) @@ -469,22 +702,21 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { test("remove executors") { val manager = createManager(createConf(5, 10, 5)) - (1 to 10).map(_.toString).foreach { id => onExecutorAdded(manager, id) } + (1 to 10).map(_.toString).foreach { id => onExecutorAddedDefaultProfile(manager, id) } // Keep removing until the limit is reached assert(executorsPendingToRemove(manager).isEmpty) - assert(removeExecutor(manager, "1")) + assert(removeExecutorDefaultProfile(manager, "1")) assert(executorsPendingToRemove(manager).size === 1) assert(executorsPendingToRemove(manager).contains("1")) - assert(removeExecutor(manager, "2")) - assert(removeExecutor(manager, "3")) + assert(removeExecutorDefaultProfile(manager, "2")) + assert(removeExecutorDefaultProfile(manager, "3")) assert(executorsPendingToRemove(manager).size === 3) assert(executorsPendingToRemove(manager).contains("2")) assert(executorsPendingToRemove(manager).contains("3")) - assert(executorsPendingToRemove(manager).size === 3) - assert(removeExecutor(manager, "4")) - assert(removeExecutor(manager, "5")) - assert(!removeExecutor(manager, "6")) // reached the limit of 5 + assert(removeExecutorDefaultProfile(manager, "4")) + assert(removeExecutorDefaultProfile(manager, "5")) + assert(!removeExecutorDefaultProfile(manager, "6")) // reached the limit of 5 assert(executorsPendingToRemove(manager).size === 5) assert(executorsPendingToRemove(manager).contains("4")) assert(executorsPendingToRemove(manager).contains("5")) @@ -508,29 +740,29 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { // Try removing again // This should still fail because the number pending + running is still at the limit - assert(!removeExecutor(manager, "7")) + assert(!removeExecutorDefaultProfile(manager, "7")) assert(executorsPendingToRemove(manager).isEmpty) - assert(!removeExecutor(manager, "8")) + assert(!removeExecutorDefaultProfile(manager, "8")) assert(executorsPendingToRemove(manager).isEmpty) } test("remove multiple executors") { val manager = createManager(createConf(5, 10, 5)) - (1 to 10).map(_.toString).foreach { id => onExecutorAdded(manager, id) } + (1 to 10).map(_.toString).foreach { id => onExecutorAddedDefaultProfile(manager, id) } // Keep removing until the limit is reached assert(executorsPendingToRemove(manager).isEmpty) - assert(removeExecutors(manager, Seq("1")) === Seq("1")) + assert(removeExecutorsDefaultProfile(manager, Seq("1")) === Seq("1")) assert(executorsPendingToRemove(manager).size === 1) assert(executorsPendingToRemove(manager).contains("1")) - assert(removeExecutors(manager, Seq("2", "3")) === Seq("2", "3")) + assert(removeExecutorsDefaultProfile(manager, Seq("2", "3")) === Seq("2", "3")) assert(executorsPendingToRemove(manager).size === 3) assert(executorsPendingToRemove(manager).contains("2")) assert(executorsPendingToRemove(manager).contains("3")) assert(executorsPendingToRemove(manager).size === 3) - assert(removeExecutor(manager, "4")) - assert(removeExecutors(manager, Seq("5")) === Seq("5")) - assert(!removeExecutor(manager, "6")) // reached the limit of 5 + assert(removeExecutorDefaultProfile(manager, "4")) + assert(removeExecutorsDefaultProfile(manager, Seq("5")) === Seq("5")) + assert(!removeExecutorDefaultProfile(manager, "6")) // reached the limit of 5 assert(executorsPendingToRemove(manager).size === 5) assert(executorsPendingToRemove(manager).contains("4")) assert(executorsPendingToRemove(manager).contains("5")) @@ -554,87 +786,100 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { // Try removing again // This should still fail because the number pending + running is still at the limit - assert(!removeExecutor(manager, "7")) + assert(!removeExecutorDefaultProfile(manager, "7")) assert(executorsPendingToRemove(manager).isEmpty) - assert(removeExecutors(manager, Seq("8")) !== Seq("8")) + assert(removeExecutorsDefaultProfile(manager, Seq("8")) !== Seq("8")) assert(executorsPendingToRemove(manager).isEmpty) } - test ("Removing with various numExecutorsTarget condition") { + test ("Removing with various numExecutorsTargetForDefaultProfileId condition") { val manager = createManager(createConf(5, 12, 5)) post(SparkListenerStageSubmitted(createStageInfo(0, 8))) - // Remove when numExecutorsTarget is the same as the current number of executors - assert(addExecutors(manager) === 1) - assert(addExecutors(manager) === 2) - (1 to 8).foreach(execId => onExecutorAdded(manager, execId.toString)) + val updatesNeeded = + new mutable.HashMap[ResourceProfile, ExecutorAllocationManager.TargetNumUpdates] + + // Remove when numExecutorsTargetForDefaultProfileId is the same as the current + // number of executors + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + (1 to 8).foreach(execId => onExecutorAddedDefaultProfile(manager, execId.toString)) (1 to 8).map { i => createTaskInfo(i, i, s"$i") }.foreach { info => post(SparkListenerTaskStart(0, 0, info)) } assert(manager.executorMonitor.executorCount === 8) - assert(numExecutorsTarget(manager) === 8) - assert(maxNumExecutorsNeeded(manager) == 8) - assert(!removeExecutor(manager, "1")) // won't work since numExecutorsTarget == numExecutors + assert(numExecutorsTargetForDefaultProfileId(manager) === 8) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 8) + // won't work since numExecutorsTargetForDefaultProfileId == numExecutors + assert(!removeExecutorDefaultProfile(manager, "1")) - // Remove executors when numExecutorsTarget is lower than current number of executors + // Remove executors when numExecutorsTargetForDefaultProfileId is lower than + // current number of executors (1 to 3).map { i => createTaskInfo(i, i, s"$i") }.foreach { info => post(SparkListenerTaskEnd(0, 0, null, Success, info, new ExecutorMetrics, null)) } adjustRequestedExecutors(manager) assert(manager.executorMonitor.executorCount === 8) - assert(numExecutorsTarget(manager) === 5) - assert(maxNumExecutorsNeeded(manager) == 5) - assert(removeExecutor(manager, "1")) - assert(removeExecutors(manager, Seq("2", "3"))=== Seq("2", "3")) + assert(numExecutorsTargetForDefaultProfileId(manager) === 5) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 5) + assert(removeExecutorDefaultProfile(manager, "1")) + assert(removeExecutorsDefaultProfile(manager, Seq("2", "3"))=== Seq("2", "3")) onExecutorRemoved(manager, "1") onExecutorRemoved(manager, "2") onExecutorRemoved(manager, "3") - // numExecutorsTarget is lower than minNumExecutors + // numExecutorsTargetForDefaultProfileId is lower than minNumExecutors post(SparkListenerTaskEnd(0, 0, null, Success, createTaskInfo(4, 4, "4"), new ExecutorMetrics, null)) assert(manager.executorMonitor.executorCount === 5) - assert(numExecutorsTarget(manager) === 5) - assert(maxNumExecutorsNeeded(manager) == 4) - assert(!removeExecutor(manager, "4")) // lower limit - assert(addExecutors(manager) === 0) // upper limit + assert(numExecutorsTargetForDefaultProfileId(manager) === 5) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) == 4) + assert(!removeExecutorDefaultProfile(manager, "4")) // lower limit + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 0) // upper limit } test ("interleaving add and remove") { val manager = createManager(createConf(5, 12, 5)) post(SparkListenerStageSubmitted(createStageInfo(0, 1000))) + val updatesNeeded = + new mutable.HashMap[ResourceProfile, ExecutorAllocationManager.TargetNumUpdates] + // Add a few executors - assert(addExecutors(manager) === 1) - assert(addExecutors(manager) === 2) - onExecutorAdded(manager, "1") - onExecutorAdded(manager, "2") - onExecutorAdded(manager, "3") - onExecutorAdded(manager, "4") - onExecutorAdded(manager, "5") - onExecutorAdded(manager, "6") - onExecutorAdded(manager, "7") - onExecutorAdded(manager, "8") + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + onExecutorAddedDefaultProfile(manager, "1") + onExecutorAddedDefaultProfile(manager, "2") + onExecutorAddedDefaultProfile(manager, "3") + onExecutorAddedDefaultProfile(manager, "4") + onExecutorAddedDefaultProfile(manager, "5") + onExecutorAddedDefaultProfile(manager, "6") + onExecutorAddedDefaultProfile(manager, "7") + onExecutorAddedDefaultProfile(manager, "8") assert(manager.executorMonitor.executorCount === 8) - assert(numExecutorsTarget(manager) === 8) + assert(numExecutorsTargetForDefaultProfileId(manager) === 8) // Remove when numTargetExecutors is equal to the current number of executors - assert(!removeExecutor(manager, "1")) - assert(removeExecutors(manager, Seq("2", "3")) !== Seq("2", "3")) + assert(!removeExecutorDefaultProfile(manager, "1")) + assert(removeExecutorsDefaultProfile(manager, Seq("2", "3")) !== Seq("2", "3")) // Remove until limit - onExecutorAdded(manager, "9") - onExecutorAdded(manager, "10") - onExecutorAdded(manager, "11") - onExecutorAdded(manager, "12") + onExecutorAddedDefaultProfile(manager, "9") + onExecutorAddedDefaultProfile(manager, "10") + onExecutorAddedDefaultProfile(manager, "11") + onExecutorAddedDefaultProfile(manager, "12") assert(manager.executorMonitor.executorCount === 12) - assert(numExecutorsTarget(manager) === 8) + assert(numExecutorsTargetForDefaultProfileId(manager) === 8) - assert(removeExecutor(manager, "1")) - assert(removeExecutors(manager, Seq("2", "3", "4")) === Seq("2", "3", "4")) - assert(!removeExecutor(manager, "5")) // lower limit reached - assert(!removeExecutor(manager, "6")) + assert(removeExecutorDefaultProfile(manager, "1")) + assert(removeExecutorsDefaultProfile(manager, Seq("2", "3", "4")) === Seq("2", "3", "4")) + assert(!removeExecutorDefaultProfile(manager, "5")) // lower limit reached + assert(!removeExecutorDefaultProfile(manager, "6")) onExecutorRemoved(manager, "1") onExecutorRemoved(manager, "2") onExecutorRemoved(manager, "3") @@ -642,33 +887,36 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { assert(manager.executorMonitor.executorCount === 8) // Add until limit - assert(!removeExecutor(manager, "7")) // still at lower limit + assert(!removeExecutorDefaultProfile(manager, "7")) // still at lower limit assert((manager, Seq("8")) !== Seq("8")) - onExecutorAdded(manager, "13") - onExecutorAdded(manager, "14") - onExecutorAdded(manager, "15") - onExecutorAdded(manager, "16") + onExecutorAddedDefaultProfile(manager, "13") + onExecutorAddedDefaultProfile(manager, "14") + onExecutorAddedDefaultProfile(manager, "15") + onExecutorAddedDefaultProfile(manager, "16") assert(manager.executorMonitor.executorCount === 12) // Remove succeeds again, now that we are no longer at the lower limit - assert(removeExecutors(manager, Seq("5", "6", "7")) === Seq("5", "6", "7")) - assert(removeExecutor(manager, "8")) + assert(removeExecutorsDefaultProfile(manager, Seq("5", "6", "7")) === Seq("5", "6", "7")) + assert(removeExecutorDefaultProfile(manager, "8")) assert(manager.executorMonitor.executorCount === 12) onExecutorRemoved(manager, "5") onExecutorRemoved(manager, "6") assert(manager.executorMonitor.executorCount === 10) - assert(numExecutorsToAdd(manager) === 4) + assert(numExecutorsToAddForDefaultProfile(manager) === 4) onExecutorRemoved(manager, "9") onExecutorRemoved(manager, "10") - assert(addExecutors(manager) === 4) // at upper limit - onExecutorAdded(manager, "17") - onExecutorAdded(manager, "18") + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 4) // at upper limit + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + onExecutorAddedDefaultProfile(manager, "17") + onExecutorAddedDefaultProfile(manager, "18") assert(manager.executorMonitor.executorCount === 10) - assert(addExecutors(manager) === 0) // still at upper limit - onExecutorAdded(manager, "19") - onExecutorAdded(manager, "20") + // still at upper limit + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 0) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + onExecutorAddedDefaultProfile(manager, "19") + onExecutorAddedDefaultProfile(manager, "20") assert(manager.executorMonitor.executorCount === 12) - assert(numExecutorsTarget(manager) === 12) + assert(numExecutorsTargetForDefaultProfileId(manager) === 12) } test("starting/canceling add timer") { @@ -706,22 +954,22 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { val manager = createManager(createConf(0, 20, 0), clock = clock) // No events - we should not be adding or removing - assert(numExecutorsTarget(manager) === 0) + assert(numExecutorsTargetForDefaultProfileId(manager) === 0) assert(executorsPendingToRemove(manager).isEmpty) schedule(manager) - assert(numExecutorsTarget(manager) === 0) + assert(numExecutorsTargetForDefaultProfileId(manager) === 0) assert(executorsPendingToRemove(manager).isEmpty) clock.advance(100L) schedule(manager) - assert(numExecutorsTarget(manager) === 0) + assert(numExecutorsTargetForDefaultProfileId(manager) === 0) assert(executorsPendingToRemove(manager).isEmpty) clock.advance(1000L) schedule(manager) - assert(numExecutorsTarget(manager) === 0) + assert(numExecutorsTargetForDefaultProfileId(manager) === 0) assert(executorsPendingToRemove(manager).isEmpty) clock.advance(10000L) schedule(manager) - assert(numExecutorsTarget(manager) === 0) + assert(numExecutorsTargetForDefaultProfileId(manager) === 0) assert(executorsPendingToRemove(manager).isEmpty) } @@ -734,43 +982,43 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { onSchedulerBacklogged(manager) clock.advance(schedulerBacklogTimeout * 1000 / 2) schedule(manager) - assert(numExecutorsTarget(manager) === 0) // timer not exceeded yet + assert(numExecutorsTargetForDefaultProfileId(manager) === 0) // timer not exceeded yet clock.advance(schedulerBacklogTimeout * 1000) schedule(manager) - assert(numExecutorsTarget(manager) === 1) // first timer exceeded + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) // first timer exceeded clock.advance(sustainedSchedulerBacklogTimeout * 1000 / 2) schedule(manager) - assert(numExecutorsTarget(manager) === 1) // second timer not exceeded yet + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) // second timer not exceeded yet clock.advance(sustainedSchedulerBacklogTimeout * 1000) schedule(manager) - assert(numExecutorsTarget(manager) === 1 + 2) // second timer exceeded + assert(numExecutorsTargetForDefaultProfileId(manager) === 1 + 2) // second timer exceeded clock.advance(sustainedSchedulerBacklogTimeout * 1000) schedule(manager) - assert(numExecutorsTarget(manager) === 1 + 2 + 4) // third timer exceeded + assert(numExecutorsTargetForDefaultProfileId(manager) === 1 + 2 + 4) // third timer exceeded // Scheduler queue drained onSchedulerQueueEmpty(manager) clock.advance(sustainedSchedulerBacklogTimeout * 1000) schedule(manager) - assert(numExecutorsTarget(manager) === 7) // timer is canceled + assert(numExecutorsTargetForDefaultProfileId(manager) === 7) // timer is canceled clock.advance(sustainedSchedulerBacklogTimeout * 1000) schedule(manager) - assert(numExecutorsTarget(manager) === 7) + assert(numExecutorsTargetForDefaultProfileId(manager) === 7) // Scheduler queue backlogged again onSchedulerBacklogged(manager) clock.advance(schedulerBacklogTimeout * 1000) schedule(manager) - assert(numExecutorsTarget(manager) === 7 + 1) // timer restarted + assert(numExecutorsTargetForDefaultProfileId(manager) === 7 + 1) // timer restarted clock.advance(sustainedSchedulerBacklogTimeout * 1000) schedule(manager) - assert(numExecutorsTarget(manager) === 7 + 1 + 2) + assert(numExecutorsTargetForDefaultProfileId(manager) === 7 + 1 + 2) clock.advance(sustainedSchedulerBacklogTimeout * 1000) schedule(manager) - assert(numExecutorsTarget(manager) === 7 + 1 + 2 + 4) + assert(numExecutorsTargetForDefaultProfileId(manager) === 7 + 1 + 2 + 4) clock.advance(sustainedSchedulerBacklogTimeout * 1000) schedule(manager) - assert(numExecutorsTarget(manager) === 20) // limit reached + assert(numExecutorsTargetForDefaultProfileId(manager) === 20) // limit reached } test("mock polling loop remove behavior") { @@ -778,9 +1026,9 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { val manager = createManager(createConf(1, 20, 1), clock = clock) // Remove idle executors on timeout - onExecutorAdded(manager, "executor-1") - onExecutorAdded(manager, "executor-2") - onExecutorAdded(manager, "executor-3") + onExecutorAddedDefaultProfile(manager, "executor-1") + onExecutorAddedDefaultProfile(manager, "executor-2") + onExecutorAddedDefaultProfile(manager, "executor-3") assert(executorsPendingToRemove(manager).isEmpty) // idle threshold not reached yet @@ -796,10 +1044,10 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { assert(executorsPendingToRemove(manager).size === 2) // limit reached (1 executor remaining) // Mark a subset as busy - only idle executors should be removed - onExecutorAdded(manager, "executor-4") - onExecutorAdded(manager, "executor-5") - onExecutorAdded(manager, "executor-6") - onExecutorAdded(manager, "executor-7") + onExecutorAddedDefaultProfile(manager, "executor-4") + onExecutorAddedDefaultProfile(manager, "executor-5") + onExecutorAddedDefaultProfile(manager, "executor-6") + onExecutorAddedDefaultProfile(manager, "executor-7") assert(manager.executorMonitor.executorCount === 7) assert(executorsPendingToRemove(manager).size === 2) // 2 pending to be removed onExecutorBusy(manager, "executor-4") @@ -864,23 +1112,31 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { val stage1 = createStageInfo(0, 1000) post(SparkListenerStageSubmitted(stage1)) - assert(addExecutors(manager) === 1) - assert(addExecutors(manager) === 2) - assert(addExecutors(manager) === 4) - assert(addExecutors(manager) === 8) - assert(numExecutorsTarget(manager) === 15) + val updatesNeeded = + new mutable.HashMap[ResourceProfile, ExecutorAllocationManager.TargetNumUpdates] + + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 4) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 8) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 15) (0 until 15).foreach { i => - onExecutorAdded(manager, s"executor-$i") + onExecutorAddedDefaultProfile(manager, s"executor-$i") } assert(manager.executorMonitor.executorCount === 15) post(SparkListenerStageCompleted(stage1)) adjustRequestedExecutors(manager) - assert(numExecutorsTarget(manager) === 0) + assert(numExecutorsTargetForDefaultProfileId(manager) === 0) post(SparkListenerStageSubmitted(createStageInfo(1, 1000))) - addExecutors(manager) - assert(numExecutorsTarget(manager) === 16) + addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 16) } test("avoid ramp down initial executors until first job is submitted") { @@ -888,19 +1144,19 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { val manager = createManager(createConf(2, 5, 3), clock = clock) // Verify the initial number of executors - assert(numExecutorsTarget(manager) === 3) + assert(numExecutorsTargetForDefaultProfileId(manager) === 3) schedule(manager) // Verify whether the initial number of executors is kept with no pending tasks - assert(numExecutorsTarget(manager) === 3) + assert(numExecutorsTargetForDefaultProfileId(manager) === 3) post(SparkListenerStageSubmitted(createStageInfo(1, 2))) clock.advance(100L) - assert(maxNumExecutorsNeeded(manager) === 2) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) === 2) schedule(manager) // Verify that current number of executors should be ramp down when first job is submitted - assert(numExecutorsTarget(manager) === 2) + assert(numExecutorsTargetForDefaultProfileId(manager) === 2) } test("avoid ramp down initial executors until idle executor is timeout") { @@ -908,20 +1164,20 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { val manager = createManager(createConf(2, 5, 3), clock = clock) // Verify the initial number of executors - assert(numExecutorsTarget(manager) === 3) + assert(numExecutorsTargetForDefaultProfileId(manager) === 3) schedule(manager) // Verify the initial number of executors is kept when no pending tasks - assert(numExecutorsTarget(manager) === 3) + assert(numExecutorsTargetForDefaultProfileId(manager) === 3) (0 until 3).foreach { i => - onExecutorAdded(manager, s"executor-$i") + onExecutorAddedDefaultProfile(manager, s"executor-$i") } clock.advance(executorIdleTimeout * 1000) - assert(maxNumExecutorsNeeded(manager) === 0) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) === 0) schedule(manager) - // Verify executor is timeout,numExecutorsTarget is recalculated - assert(numExecutorsTarget(manager) === 2) + // Verify executor is timeout,numExecutorsTargetForDefaultProfileId is recalculated + assert(numExecutorsTargetForDefaultProfileId(manager) === 2) } test("get pending task number and related locality preference") { @@ -937,7 +1193,8 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { val stageInfo1 = createStageInfo(1, 5, localityPreferences1) post(SparkListenerStageSubmitted(stageInfo1)) - assert(localityAwareTasks(manager) === 3) + assert(localityAwareTasksForDefaultProfile(manager) === 3) + val hostToLocal = hostToLocalTaskCount(manager) assert(hostToLocalTaskCount(manager) === Map("host1" -> 2, "host2" -> 3, "host3" -> 2, "host4" -> 2)) @@ -949,67 +1206,76 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { val stageInfo2 = createStageInfo(2, 3, localityPreferences2) post(SparkListenerStageSubmitted(stageInfo2)) - assert(localityAwareTasks(manager) === 5) + assert(localityAwareTasksForDefaultProfile(manager) === 5) assert(hostToLocalTaskCount(manager) === Map("host1" -> 2, "host2" -> 4, "host3" -> 4, "host4" -> 3, "host5" -> 2)) post(SparkListenerStageCompleted(stageInfo1)) - assert(localityAwareTasks(manager) === 2) + assert(localityAwareTasksForDefaultProfile(manager) === 2) assert(hostToLocalTaskCount(manager) === Map("host2" -> 1, "host3" -> 2, "host4" -> 1, "host5" -> 2)) } - test("SPARK-8366: maxNumExecutorsNeeded should properly handle failed tasks") { + test("SPARK-8366: maxNumExecutorsNeededPerResourceProfile should properly handle failed tasks") { val manager = createManager(createConf()) - assert(maxNumExecutorsNeeded(manager) === 0) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) === 0) post(SparkListenerStageSubmitted(createStageInfo(0, 1))) - assert(maxNumExecutorsNeeded(manager) === 1) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) === 1) val taskInfo = createTaskInfo(1, 1, "executor-1") post(SparkListenerTaskStart(0, 0, taskInfo)) - assert(maxNumExecutorsNeeded(manager) === 1) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) === 1) // If the task is failed, we expect it to be resubmitted later. val taskEndReason = ExceptionFailure(null, null, null, null, None) post(SparkListenerTaskEnd(0, 0, null, taskEndReason, taskInfo, new ExecutorMetrics, null)) - assert(maxNumExecutorsNeeded(manager) === 1) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) === 1) } test("reset the state of allocation manager") { val manager = createManager(createConf()) - assert(numExecutorsTarget(manager) === 1) - assert(numExecutorsToAdd(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) + + val updatesNeeded = + new mutable.HashMap[ResourceProfile, ExecutorAllocationManager.TargetNumUpdates] // Allocation manager is reset when adding executor requests are sent without reporting back // executor added. post(SparkListenerStageSubmitted(createStageInfo(0, 10))) - assert(addExecutors(manager) === 1) - assert(numExecutorsTarget(manager) === 2) - assert(addExecutors(manager) === 2) - assert(numExecutorsTarget(manager) === 4) - assert(addExecutors(manager) === 1) - assert(numExecutorsTarget(manager) === 5) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 2) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 2) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 4) + assert(addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) === 1) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 5) manager.reset() - assert(numExecutorsTarget(manager) === 1) - assert(numExecutorsToAdd(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) assert(manager.executorMonitor.executorCount === 0) // Allocation manager is reset when executors are added. post(SparkListenerStageSubmitted(createStageInfo(0, 10))) - addExecutors(manager) - addExecutors(manager) - addExecutors(manager) - assert(numExecutorsTarget(manager) === 5) - - onExecutorAdded(manager, "first") - onExecutorAdded(manager, "second") - onExecutorAdded(manager, "third") - onExecutorAdded(manager, "fourth") - onExecutorAdded(manager, "fifth") + addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 5) + + onExecutorAddedDefaultProfile(manager, "first") + onExecutorAddedDefaultProfile(manager, "second") + onExecutorAddedDefaultProfile(manager, "third") + onExecutorAddedDefaultProfile(manager, "fourth") + onExecutorAddedDefaultProfile(manager, "fifth") assert(manager.executorMonitor.executorCount === 5) // Cluster manager lost will make all the live executors lost, so here simulate this behavior @@ -1020,28 +1286,31 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { onExecutorRemoved(manager, "fifth") manager.reset() - assert(numExecutorsTarget(manager) === 1) - assert(numExecutorsToAdd(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) assert(manager.executorMonitor.executorCount === 0) // Allocation manager is reset when executors are pending to remove - addExecutors(manager) - addExecutors(manager) - addExecutors(manager) - assert(numExecutorsTarget(manager) === 5) - - onExecutorAdded(manager, "first") - onExecutorAdded(manager, "second") - onExecutorAdded(manager, "third") - onExecutorAdded(manager, "fourth") - onExecutorAdded(manager, "fifth") - onExecutorAdded(manager, "sixth") - onExecutorAdded(manager, "seventh") - onExecutorAdded(manager, "eighth") + addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + addExecutorsToTargetForDefaultProfile(manager, updatesNeeded) + doUpdateRequest(manager, updatesNeeded.toMap, clock.getTimeMillis()) + assert(numExecutorsTargetForDefaultProfileId(manager) === 5) + + onExecutorAddedDefaultProfile(manager, "first") + onExecutorAddedDefaultProfile(manager, "second") + onExecutorAddedDefaultProfile(manager, "third") + onExecutorAddedDefaultProfile(manager, "fourth") + onExecutorAddedDefaultProfile(manager, "fifth") + onExecutorAddedDefaultProfile(manager, "sixth") + onExecutorAddedDefaultProfile(manager, "seventh") + onExecutorAddedDefaultProfile(manager, "eighth") assert(manager.executorMonitor.executorCount === 8) - removeExecutor(manager, "first") - removeExecutors(manager, Seq("second", "third")) + removeExecutorDefaultProfile(manager, "first") + removeExecutorsDefaultProfile(manager, Seq("second", "third")) assert(executorsPendingToRemove(manager) === Set("first", "second", "third")) assert(manager.executorMonitor.executorCount === 8) @@ -1055,8 +1324,8 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { manager.reset() - assert(numExecutorsTarget(manager) === 1) - assert(numExecutorsToAdd(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) + assert(numExecutorsToAddForDefaultProfile(manager) === 1) assert(executorsPendingToRemove(manager) === Set.empty) assert(manager.executorMonitor.executorCount === 0) } @@ -1067,31 +1336,31 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { createConf(1, 2, 1).set(config.DYN_ALLOCATION_TESTING, false), clock = clock) - when(client.requestTotalExecutors(meq(2), any(), any())).thenReturn(true) + when(client.requestTotalExecutors(any(), any(), any())).thenReturn(true) // test setup -- job with 2 tasks, scale up to two executors - assert(numExecutorsTarget(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) post(SparkListenerExecutorAdded( clock.getTimeMillis(), "executor-1", new ExecutorInfo("host1", 1, Map.empty, Map.empty))) post(SparkListenerStageSubmitted(createStageInfo(0, 2))) clock.advance(1000) manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) - assert(numExecutorsTarget(manager) === 2) + assert(numExecutorsTargetForDefaultProfileId(manager) === 2) val taskInfo0 = createTaskInfo(0, 0, "executor-1") post(SparkListenerTaskStart(0, 0, taskInfo0)) post(SparkListenerExecutorAdded( clock.getTimeMillis(), "executor-2", new ExecutorInfo("host1", 1, Map.empty, Map.empty))) val taskInfo1 = createTaskInfo(1, 1, "executor-2") post(SparkListenerTaskStart(0, 0, taskInfo1)) - assert(numExecutorsTarget(manager) === 2) + assert(numExecutorsTargetForDefaultProfileId(manager) === 2) // have one task finish -- we should adjust the target number of executors down // but we should *not* kill any executors yet post(SparkListenerTaskEnd(0, 0, null, Success, taskInfo0, new ExecutorMetrics, null)) - assert(maxNumExecutorsNeeded(manager) === 1) - assert(numExecutorsTarget(manager) === 2) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 2) clock.advance(1000) manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.nanoTime()) - assert(numExecutorsTarget(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) verify(client, never).killExecutors(any(), any(), any(), any()) // now we cross the idle timeout for executor-1, so we kill it. the really important @@ -1101,8 +1370,8 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { .thenReturn(Seq("executor-1")) clock.advance(3000) schedule(manager) - assert(maxNumExecutorsNeeded(manager) === 1) - assert(numExecutorsTarget(manager) === 1) + assert(maxNumExecutorsNeededPerResourceProfile(manager, defaultProfile) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) // here's the important verify -- we did kill the executors, but did not adjust the target count verify(client).killExecutors(Seq("executor-1"), false, false, false) } @@ -1110,7 +1379,7 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { test("SPARK-26758 check executor target number after idle time out ") { val clock = new ManualClock(10000L) val manager = createManager(createConf(1, 5, 3), clock = clock) - assert(numExecutorsTarget(manager) === 3) + assert(numExecutorsTargetForDefaultProfileId(manager) === 3) post(SparkListenerExecutorAdded( clock.getTimeMillis(), "executor-1", new ExecutorInfo("host1", 1, Map.empty))) post(SparkListenerExecutorAdded( @@ -1121,14 +1390,14 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { clock.advance(executorIdleTimeout * 1000) schedule(manager) // once the schedule is run target executor number should be 1 - assert(numExecutorsTarget(manager) === 1) + assert(numExecutorsTargetForDefaultProfileId(manager) === 1) } private def createConf( minExecutors: Int = 1, maxExecutors: Int = 5, initialExecutors: Int = 1): SparkConf = { - new SparkConf() + val sparkConf = new SparkConf() .set(config.DYN_ALLOCATION_ENABLED, true) .set(config.DYN_ALLOCATION_MIN_EXECUTORS, minExecutors) .set(config.DYN_ALLOCATION_MAX_EXECUTORS, maxExecutors) @@ -1143,12 +1412,16 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { // SPARK-22864: effectively disable the allocation schedule by setting the period to a // really long value. .set(TEST_SCHEDULE_INTERVAL, 10000L) + sparkConf } private def createManager( conf: SparkConf, clock: Clock = new SystemClock()): ExecutorAllocationManager = { - val manager = new ExecutorAllocationManager(client, listenerBus, conf, clock = clock) + ResourceProfile.reInitDefaultProfile(conf) + rpManager = new ResourceProfileManager(conf) + val manager = new ExecutorAllocationManager(client, listenerBus, conf, clock = clock, + resourceProfileManager = rpManager) managers += manager manager.start() manager @@ -1157,7 +1430,18 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { private val execInfo = new ExecutorInfo("host1", 1, Map.empty, Map.empty, Map.empty, DEFAULT_RESOURCE_PROFILE_ID) - private def onExecutorAdded(manager: ExecutorAllocationManager, id: String): Unit = { + private def onExecutorAddedDefaultProfile( + manager: ExecutorAllocationManager, + id: String): Unit = { + post(SparkListenerExecutorAdded(0L, id, execInfo)) + } + + private def onExecutorAdded( + manager: ExecutorAllocationManager, + id: String, + rp: ResourceProfile): Unit = { + val cores = rp.getExecutorCores.getOrElse(1) + val execInfo = new ExecutorInfo("host1", cores, Map.empty, Map.empty, Map.empty, rp.id) post(SparkListenerExecutorAdded(0L, id, execInfo)) } @@ -1176,8 +1460,18 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite { post(SparkListenerTaskEnd(1, 1, "foo", Success, info, new ExecutorMetrics, null)) } - private def removeExecutor(manager: ExecutorAllocationManager, executorId: String): Boolean = { - val executorsRemoved = removeExecutors(manager, Seq(executorId)) + private def removeExecutorDefaultProfile( + manager: ExecutorAllocationManager, + executorId: String): Boolean = { + val executorsRemoved = removeExecutorsDefaultProfile(manager, Seq(executorId)) + executorsRemoved.nonEmpty && executorsRemoved(0) == executorId + } + + private def removeExecutor( + manager: ExecutorAllocationManager, + executorId: String, + rpId: Int): Boolean = { + val executorsRemoved = removeExecutors(manager, Seq((executorId, rpId))) executorsRemoved.nonEmpty && executorsRemoved(0) == executorId } @@ -1199,10 +1493,11 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester { stageId: Int, numTasks: Int, taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty, - attemptId: Int = 0 + attemptId: Int = 0, + rp: ResourceProfile = defaultProfile ): StageInfo = { new StageInfo(stageId, attemptId, "name", numTasks, Seq.empty, Seq.empty, "no details", - taskLocalityPreferences = taskLocalityPreferences) + taskLocalityPreferences = taskLocalityPreferences, resourceProfileId = rp.id) } private def createTaskInfo( @@ -1217,54 +1512,117 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester { | Helper methods for accessing private methods and fields | * ------------------------------------------------------- */ - private val _numExecutorsToAdd = PrivateMethod[Int](Symbol("numExecutorsToAdd")) - private val _numExecutorsTarget = PrivateMethod[Int](Symbol("numExecutorsTarget")) - private val _maxNumExecutorsNeeded = PrivateMethod[Int](Symbol("maxNumExecutorsNeeded")) + private val _numExecutorsToAddPerResourceProfileId = + PrivateMethod[mutable.HashMap[Int, Int]]( + Symbol("numExecutorsToAddPerResourceProfileId")) + private val _numExecutorsTargetPerResourceProfileId = + PrivateMethod[mutable.HashMap[Int, Int]]( + Symbol("numExecutorsTargetPerResourceProfileId")) + private val _maxNumExecutorsNeededPerResourceProfile = + PrivateMethod[Int](Symbol("maxNumExecutorsNeededPerResourceProfile")) private val _addTime = PrivateMethod[Long](Symbol("addTime")) private val _schedule = PrivateMethod[Unit](Symbol("schedule")) - private val _addExecutors = PrivateMethod[Int](Symbol("addExecutors")) + private val _doUpdateRequest = PrivateMethod[Unit](Symbol("doUpdateRequest")) private val _updateAndSyncNumExecutorsTarget = PrivateMethod[Int](Symbol("updateAndSyncNumExecutorsTarget")) + private val _addExecutorsToTarget = PrivateMethod[Int](Symbol("addExecutorsToTarget")) private val _removeExecutors = PrivateMethod[Seq[String]](Symbol("removeExecutors")) private val _onSchedulerBacklogged = PrivateMethod[Unit](Symbol("onSchedulerBacklogged")) private val _onSchedulerQueueEmpty = PrivateMethod[Unit](Symbol("onSchedulerQueueEmpty")) - private val _localityAwareTasks = PrivateMethod[Int](Symbol("localityAwareTasks")) - private val _hostToLocalTaskCount = - PrivateMethod[Map[String, Int]](Symbol("hostToLocalTaskCount")) + private val _localityAwareTasksPerResourceProfileId = + PrivateMethod[mutable.HashMap[Int, Int]](Symbol("numLocalityAwareTasksPerResourceProfileId")) + private val _rpIdToHostToLocalTaskCount = + PrivateMethod[Map[Int, Map[String, Int]]](Symbol("rpIdToHostToLocalTaskCount")) private val _onSpeculativeTaskSubmitted = PrivateMethod[Unit](Symbol("onSpeculativeTaskSubmitted")) - private val _totalRunningTasks = PrivateMethod[Int](Symbol("totalRunningTasks")) + private val _totalRunningTasksPerResourceProfile = + PrivateMethod[Int](Symbol("totalRunningTasksPerResourceProfile")) + + private val defaultProfile = ResourceProfile.getOrCreateDefaultProfile(new SparkConf) + + private def numExecutorsToAddForDefaultProfile(manager: ExecutorAllocationManager): Int = { + numExecutorsToAdd(manager, defaultProfile) + } + + private def numExecutorsToAdd( + manager: ExecutorAllocationManager, + rp: ResourceProfile): Int = { + val nmap = manager invokePrivate _numExecutorsToAddPerResourceProfileId() + nmap(rp.id) + } + + private def updateAndSyncNumExecutorsTarget( + manager: ExecutorAllocationManager, + now: Long): Unit = { + manager invokePrivate _updateAndSyncNumExecutorsTarget(now) + } + + private def numExecutorsTargetForDefaultProfileId(manager: ExecutorAllocationManager): Int = { + numExecutorsTarget(manager, defaultProfile.id) + } - private def numExecutorsToAdd(manager: ExecutorAllocationManager): Int = { - manager invokePrivate _numExecutorsToAdd() + private def numExecutorsTarget( + manager: ExecutorAllocationManager, + rpId: Int): Int = { + val numMap = manager invokePrivate _numExecutorsTargetPerResourceProfileId() + numMap(rpId) } - private def numExecutorsTarget(manager: ExecutorAllocationManager): Int = { - manager invokePrivate _numExecutorsTarget() + private def addExecutorsToTargetForDefaultProfile( + manager: ExecutorAllocationManager, + updatesNeeded: mutable.HashMap[ResourceProfile, + ExecutorAllocationManager.TargetNumUpdates] + ): Int = { + addExecutorsToTarget(manager, updatesNeeded, defaultProfile) + } + + private def addExecutorsToTarget( + manager: ExecutorAllocationManager, + updatesNeeded: mutable.HashMap[ResourceProfile, + ExecutorAllocationManager.TargetNumUpdates], + rp: ResourceProfile + ): Int = { + val maxNumExecutorsNeeded = + manager invokePrivate _maxNumExecutorsNeededPerResourceProfile(rp.id) + manager invokePrivate + _addExecutorsToTarget(maxNumExecutorsNeeded, rp.id, updatesNeeded) } private def addTime(manager: ExecutorAllocationManager): Long = { manager invokePrivate _addTime() } - private def schedule(manager: ExecutorAllocationManager): Unit = { - manager invokePrivate _schedule() + private def doUpdateRequest( + manager: ExecutorAllocationManager, + updates: Map[ResourceProfile, ExecutorAllocationManager.TargetNumUpdates], + now: Long): Unit = { + manager invokePrivate _doUpdateRequest(updates, now) } - private def maxNumExecutorsNeeded(manager: ExecutorAllocationManager): Int = { - manager invokePrivate _maxNumExecutorsNeeded() + private def schedule(manager: ExecutorAllocationManager): Unit = { + manager invokePrivate _schedule() } - private def addExecutors(manager: ExecutorAllocationManager): Int = { - val maxNumExecutorsNeeded = manager invokePrivate _maxNumExecutorsNeeded() - manager invokePrivate _addExecutors(maxNumExecutorsNeeded) + private def maxNumExecutorsNeededPerResourceProfile( + manager: ExecutorAllocationManager, + rp: ResourceProfile): Int = { + manager invokePrivate _maxNumExecutorsNeededPerResourceProfile(rp.id) } private def adjustRequestedExecutors(manager: ExecutorAllocationManager): Int = { manager invokePrivate _updateAndSyncNumExecutorsTarget(0L) } - private def removeExecutors(manager: ExecutorAllocationManager, ids: Seq[String]): Seq[String] = { + private def removeExecutorsDefaultProfile( + manager: ExecutorAllocationManager, + ids: Seq[String]): Seq[String] = { + val idsAndProfileIds = ids.map((_, defaultProfile.id)) + manager invokePrivate _removeExecutors(idsAndProfileIds) + } + + private def removeExecutors( + manager: ExecutorAllocationManager, + ids: Seq[(String, Int)]): Seq[String] = { manager invokePrivate _removeExecutors(ids) } @@ -1280,15 +1638,22 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester { manager invokePrivate _onSpeculativeTaskSubmitted(id) } - private def localityAwareTasks(manager: ExecutorAllocationManager): Int = { - manager invokePrivate _localityAwareTasks() + private def localityAwareTasksForDefaultProfile(manager: ExecutorAllocationManager): Int = { + val localMap = manager invokePrivate _localityAwareTasksPerResourceProfileId() + localMap(defaultProfile.id) + } + + private def totalRunningTasksPerResourceProfile(manager: ExecutorAllocationManager): Int = { + manager invokePrivate _totalRunningTasksPerResourceProfile(defaultProfile.id) } - private def totalRunningTasks(manager: ExecutorAllocationManager): Int = { - manager invokePrivate _totalRunningTasks() + private def hostToLocalTaskCount( + manager: ExecutorAllocationManager): Map[String, Int] = { + val rpIdToHostLocal = manager invokePrivate _rpIdToHostToLocalTaskCount() + rpIdToHostLocal(defaultProfile.id) } - private def hostToLocalTaskCount(manager: ExecutorAllocationManager): Map[String, Int] = { - manager invokePrivate _hostToLocalTaskCount() + private def getResourceProfileIdOfExecutor(manager: ExecutorAllocationManager): Int = { + defaultProfile.id } } diff --git a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala index ff0f2f9134ed3..312691302b064 100644 --- a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala +++ b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala @@ -30,7 +30,7 @@ import org.scalatest.concurrent.Eventually._ import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.internal.config.DYN_ALLOCATION_TESTING -import org.apache.spark.resource.ResourceProfile +import org.apache.spark.resource.{ResourceProfile, ResourceProfileManager} import org.apache.spark.rpc.{RpcCallContext, RpcEndpoint, RpcEndpointRef, RpcEnv} import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ @@ -61,6 +61,7 @@ class HeartbeatReceiverSuite PrivateMethod[collection.Map[String, Long]](Symbol("executorLastSeen")) private val _executorTimeoutMs = PrivateMethod[Long](Symbol("executorTimeoutMs")) private val _killExecutorThread = PrivateMethod[ExecutorService](Symbol("killExecutorThread")) + var conf: SparkConf = _ /** * Before each test, set up the SparkContext and a custom [[HeartbeatReceiver]] @@ -68,7 +69,7 @@ class HeartbeatReceiverSuite */ override def beforeEach(): Unit = { super.beforeEach() - val conf = new SparkConf() + conf = new SparkConf() .setMaster("local[2]") .setAppName("test") .set(DYN_ALLOCATION_TESTING, true) @@ -76,7 +77,6 @@ class HeartbeatReceiverSuite scheduler = mock(classOf[TaskSchedulerImpl]) when(sc.taskScheduler).thenReturn(scheduler) when(scheduler.nodeBlacklist).thenReturn(Predef.Set[String]()) - when(scheduler.resourcesReqsPerTask).thenReturn(Seq.empty) when(scheduler.sc).thenReturn(sc) heartbeatReceiverClock = new ManualClock heartbeatReceiver = new HeartbeatReceiver(sc, heartbeatReceiverClock) @@ -164,9 +164,10 @@ class HeartbeatReceiverSuite test("expire dead hosts should kill executors with replacement (SPARK-8119)") { // Set up a fake backend and cluster manager to simulate killing executors val rpcEnv = sc.env.rpcEnv - val fakeClusterManager = new FakeClusterManager(rpcEnv) + val fakeClusterManager = new FakeClusterManager(rpcEnv, conf) val fakeClusterManagerRef = rpcEnv.setupEndpoint("fake-cm", fakeClusterManager) - val fakeSchedulerBackend = new FakeSchedulerBackend(scheduler, rpcEnv, fakeClusterManagerRef) + val fakeSchedulerBackend = + new FakeSchedulerBackend(scheduler, rpcEnv, fakeClusterManagerRef, sc.resourceProfileManager) when(sc.schedulerBackend).thenReturn(fakeSchedulerBackend) // Register fake executors with our fake scheduler backend @@ -282,13 +283,16 @@ private class FakeExecutorEndpoint(override val rpcEnv: RpcEnv) extends RpcEndpo private class FakeSchedulerBackend( scheduler: TaskSchedulerImpl, rpcEnv: RpcEnv, - clusterManagerEndpoint: RpcEndpointRef) + clusterManagerEndpoint: RpcEndpointRef, + resourceProfileManager: ResourceProfileManager) extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) { - protected override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = { + protected override def doRequestTotalExecutors( + resourceProfileToTotalExecs: Map[ResourceProfile, Int]): Future[Boolean] = { clusterManagerEndpoint.ask[Boolean]( - RequestExecutors(requestedTotal, localityAwareTasks, hostToLocalTaskCount, Set.empty)) - } + RequestExecutors(resourceProfileToTotalExecs, numLocalityAwareTasksPerResourceProfileId, + rpHostToLocalTaskCount, Set.empty)) +} protected override def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = { clusterManagerEndpoint.ask[Boolean](KillExecutors(executorIds)) @@ -298,7 +302,7 @@ private class FakeSchedulerBackend( /** * Dummy cluster manager to simulate responses to executor allocation requests. */ -private class FakeClusterManager(override val rpcEnv: RpcEnv) extends RpcEndpoint { +private class FakeClusterManager(override val rpcEnv: RpcEnv, conf: SparkConf) extends RpcEndpoint { private var targetNumExecutors = 0 private val executorIdsToKill = new mutable.HashSet[String] @@ -306,8 +310,9 @@ private class FakeClusterManager(override val rpcEnv: RpcEnv) extends RpcEndpoin def getExecutorIdsToKill: Set[String] = executorIdsToKill.toSet override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { - case RequestExecutors(requestedTotal, _, _, _) => - targetNumExecutors = requestedTotal + case RequestExecutors(resourceProfileToTotalExecs, _, _, _) => + targetNumExecutors = + resourceProfileToTotalExecs(ResourceProfile.getOrCreateDefaultProfile(conf)) context.reply(true) case KillExecutors(executorIds) => executorIdsToKill ++= executorIds diff --git a/core/src/test/scala/org/apache/spark/LocalSparkContext.scala b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala index 1fe12e116d96e..599ea8955491f 100644 --- a/core/src/test/scala/org/apache/spark/LocalSparkContext.scala +++ b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala @@ -44,7 +44,7 @@ trait LocalSparkContext extends BeforeAndAfterEach with BeforeAndAfterAll { self def resetSparkContext(): Unit = { LocalSparkContext.stop(sc) - ResourceProfile.clearDefaultProfile + ResourceProfile.clearDefaultProfile() sc = null } diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index df9c7c5eaa368..b6dfa69015c28 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -36,6 +36,7 @@ import org.scalatest.concurrent.Eventually import org.apache.spark.TestUtils._ import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Tests._ import org.apache.spark.internal.config.UI._ import org.apache.spark.resource.ResourceAllocation import org.apache.spark.resource.ResourceUtils._ @@ -784,7 +785,7 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu } test(s"Avoid setting ${CPUS_PER_TASK.key} unreasonably (SPARK-27192)") { - val FAIL_REASON = s"has to be >= the task config: ${CPUS_PER_TASK.key}" + val FAIL_REASON = " has to be >= the number of cpus per task" Seq( ("local", 2, None), ("local[2]", 3, None), @@ -864,9 +865,8 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu sc = new SparkContext(conf) }.getMessage() - assert(error.contains("The executor resource config: spark.executor.resource.gpu.amount " + - "needs to be specified since a task requirement config: spark.task.resource.gpu.amount " + - "was specified")) + assert(error.contains("No executor resource configs were not specified for the following " + + "task configs: gpu")) } test("Test parsing resources executor config < task requirements") { @@ -880,15 +880,15 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu sc = new SparkContext(conf) }.getMessage() - assert(error.contains("The executor resource config: spark.executor.resource.gpu.amount = 1 " + - "has to be >= the requested amount in task resource config: " + - "spark.task.resource.gpu.amount = 2")) + assert(error.contains("The executor resource: gpu, amount: 1 needs to be >= the task " + + "resource request amount of 2.0")) } test("Parse resources executor config not the same multiple numbers of the task requirements") { val conf = new SparkConf() .setMaster("local-cluster[1, 1, 1024]") .setAppName("test-cluster") + conf.set(RESOURCES_WARNING_TESTING, true) conf.set(TASK_GPU_ID.amountConf, "2") conf.set(EXECUTOR_GPU_ID.amountConf, "4") @@ -897,25 +897,9 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu }.getMessage() assert(error.contains( - "The configuration of resource: gpu (exec = 4, task = 2, runnable tasks = 2) will result " + - "in wasted resources due to resource CPU limiting the number of runnable tasks per " + - "executor to: 1. Please adjust your configuration.")) - } - - test("Parse resources executor config cpus not limiting resource") { - val conf = new SparkConf() - .setMaster("local-cluster[1, 8, 1024]") - .setAppName("test-cluster") - conf.set(TASK_GPU_ID.amountConf, "2") - conf.set(EXECUTOR_GPU_ID.amountConf, "4") - - var error = intercept[IllegalArgumentException] { - sc = new SparkContext(conf) - }.getMessage() - - assert(error.contains("The number of slots on an executor has to be " + - "limited by the number of cores, otherwise you waste resources and " + - "dynamic allocation doesn't work properly")) + "The configuration of resource: gpu (exec = 4, task = 2.0/1, runnable tasks = 2) will " + + "result in wasted resources due to resource cpus limiting the number of runnable " + + "tasks per executor to: 1. Please adjust your configuration.")) } test("test resource scheduling under local-cluster mode") { diff --git a/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala b/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala index a1d3077b8fc87..a3e39d7f53728 100644 --- a/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala @@ -30,7 +30,7 @@ import org.apache.spark.deploy.{ApplicationDescription, Command} import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} import org.apache.spark.deploy.master.{ApplicationInfo, Master} import org.apache.spark.deploy.worker.Worker -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.rpc.RpcEnv import org.apache.spark.util.Utils @@ -44,13 +44,13 @@ class AppClientSuite with Eventually with ScalaFutures { private val numWorkers = 2 - private val conf = new SparkConf() - private val securityManager = new SecurityManager(conf) + private var conf: SparkConf = null private var masterRpcEnv: RpcEnv = null private var workerRpcEnvs: Seq[RpcEnv] = null private var master: Master = null private var workers: Seq[Worker] = null + private var securityManager: SecurityManager = null /** * Start the local cluster. @@ -58,6 +58,8 @@ class AppClientSuite */ override def beforeAll(): Unit = { super.beforeAll() + conf = new SparkConf().set(config.Worker.WORKER_DECOMMISSION_ENABLED.key, "true") + securityManager = new SecurityManager(conf) masterRpcEnv = RpcEnv.create(Master.SYSTEM_NAME, "localhost", 0, conf, securityManager) workerRpcEnvs = (0 until numWorkers).map { i => RpcEnv.create(Worker.SYSTEM_NAME + i, "localhost", 0, conf, securityManager) @@ -111,8 +113,23 @@ class AppClientSuite assert(apps.head.getExecutorLimit === numExecutorsRequested, s"executor request failed") } + + // Save the executor id before decommissioning so we can kill it + val application = getApplications().head + val executors = application.executors + val executorId: String = executors.head._2.fullId + + // Send a decommission self to all the workers + // Note: normally the worker would send this on their own. + workers.foreach(worker => worker.decommissionSelf()) + + // Decommissioning is async. + eventually(timeout(1.seconds), interval(10.millis)) { + // We only record decommissioning for the executor we've requested + assert(ci.listener.execDecommissionedList.size === 1) + } + // Send request to kill executor, verify request was made - val executorId: String = getApplications().head.executors.head._2.fullId whenReady( ci.client.killExecutors(Seq(executorId)), timeout(10.seconds), @@ -120,6 +137,15 @@ class AppClientSuite assert(acknowledged) } + // Verify that asking for executors on the decommissioned workers fails + whenReady( + ci.client.requestTotalExecutors(numExecutorsRequested), + timeout(10.seconds), + interval(10.millis)) { acknowledged => + assert(acknowledged) + } + assert(getApplications().head.executors.size === 0) + // Issue stop command for Client to disconnect from Master ci.client.stop() @@ -189,6 +215,7 @@ class AppClientSuite val deadReasonList = new ConcurrentLinkedQueue[String]() val execAddedList = new ConcurrentLinkedQueue[String]() val execRemovedList = new ConcurrentLinkedQueue[String]() + val execDecommissionedList = new ConcurrentLinkedQueue[String]() def connected(id: String): Unit = { connectedIdList.add(id) @@ -218,6 +245,10 @@ class AppClientSuite execRemovedList.add(id) } + def executorDecommissioned(id: String, message: String): Unit = { + execDecommissionedList.add(id) + } + def workerRemoved(workerId: String, host: String, message: String): Unit = {} } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterBuilderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterBuilderSuite.scala index 86511ae08784a..c905797bf1287 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterBuilderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/BasicEventFilterBuilderSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.deploy.history import org.apache.spark.{SparkFunSuite, Success, TaskResultLost, TaskState} import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} +import org.apache.spark.resource.ResourceProfile import org.apache.spark.scheduler._ import org.apache.spark.status.ListenerEventsTestHelper @@ -141,7 +142,8 @@ class BasicEventFilterBuilderSuite extends SparkFunSuite { // - Re-submit stage 1, all tasks, and succeed them and the stage. val oldS1 = stages.last val newS1 = new StageInfo(oldS1.stageId, oldS1.attemptNumber + 1, oldS1.name, oldS1.numTasks, - oldS1.rddInfos, oldS1.parentIds, oldS1.details, oldS1.taskMetrics) + oldS1.rddInfos, oldS1.parentIds, oldS1.details, oldS1.taskMetrics, + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) time += 1 newS1.submissionTime = Some(time) diff --git a/core/src/test/scala/org/apache/spark/internal/plugin/PluginContainerSuite.scala b/core/src/test/scala/org/apache/spark/internal/plugin/PluginContainerSuite.scala index cf2d9293ef822..7888796dd55e6 100644 --- a/core/src/test/scala/org/apache/spark/internal/plugin/PluginContainerSuite.scala +++ b/core/src/test/scala/org/apache/spark/internal/plugin/PluginContainerSuite.scala @@ -139,7 +139,7 @@ class PluginContainerSuite extends SparkFunSuite with BeforeAndAfterEach with Lo .set(NonLocalModeSparkPlugin.TEST_PATH_CONF, path.getAbsolutePath()) sc = new SparkContext(conf) - TestUtils.waitUntilExecutorsUp(sc, 2, 10000) + TestUtils.waitUntilExecutorsUp(sc, 2, 60000) eventually(timeout(10.seconds), interval(100.millis)) { val children = path.listFiles() @@ -169,7 +169,7 @@ class PluginContainerSuite extends SparkFunSuite with BeforeAndAfterEach with Lo sc = new SparkContext(conf) // Ensure all executors has started - TestUtils.waitUntilExecutorsUp(sc, 1, 10000) + TestUtils.waitUntilExecutorsUp(sc, 1, 60000) var children = Array.empty[File] eventually(timeout(10.seconds), interval(100.millis)) { diff --git a/core/src/test/scala/org/apache/spark/resource/ResourceDiscoveryPluginSuite.scala b/core/src/test/scala/org/apache/spark/resource/ResourceDiscoveryPluginSuite.scala index 7a05daa2ad715..437c903e77d4a 100644 --- a/core/src/test/scala/org/apache/spark/resource/ResourceDiscoveryPluginSuite.scala +++ b/core/src/test/scala/org/apache/spark/resource/ResourceDiscoveryPluginSuite.scala @@ -56,7 +56,7 @@ class ResourceDiscoveryPluginSuite extends SparkFunSuite with LocalSparkContext .set(EXECUTOR_FPGA_ID.amountConf, "1") sc = new SparkContext(conf) - TestUtils.waitUntilExecutorsUp(sc, 2, 10000) + TestUtils.waitUntilExecutorsUp(sc, 2, 60000) eventually(timeout(10.seconds), interval(100.millis)) { val children = dir.listFiles() @@ -84,7 +84,7 @@ class ResourceDiscoveryPluginSuite extends SparkFunSuite with LocalSparkContext .set(SPARK_RESOURCES_DIR, dir.getName()) sc = new SparkContext(conf) - TestUtils.waitUntilExecutorsUp(sc, 2, 10000) + TestUtils.waitUntilExecutorsUp(sc, 2, 60000) eventually(timeout(10.seconds), interval(100.millis)) { val children = dir.listFiles() @@ -111,7 +111,7 @@ class ResourceDiscoveryPluginSuite extends SparkFunSuite with LocalSparkContext .set(SPARK_RESOURCES_DIR, dir.getName()) sc = new SparkContext(conf) - TestUtils.waitUntilExecutorsUp(sc, 2, 10000) + TestUtils.waitUntilExecutorsUp(sc, 2, 60000) eventually(timeout(10.seconds), interval(100.millis)) { val children = dir.listFiles() @@ -137,7 +137,7 @@ class ResourceDiscoveryPluginSuite extends SparkFunSuite with LocalSparkContext .set(SPARK_RESOURCES_DIR, dir.getName()) sc = new SparkContext(conf) - TestUtils.waitUntilExecutorsUp(sc, 2, 10000) + TestUtils.waitUntilExecutorsUp(sc, 2, 60000) assert(sc.resources.size === 1) assert(sc.resources.get(GPU).get.addresses === Array("5", "6")) diff --git a/core/src/test/scala/org/apache/spark/resource/ResourceProfileManagerSuite.scala b/core/src/test/scala/org/apache/spark/resource/ResourceProfileManagerSuite.scala new file mode 100644 index 0000000000000..075260317284d --- /dev/null +++ b/core/src/test/scala/org/apache/spark/resource/ResourceProfileManagerSuite.scala @@ -0,0 +1,103 @@ +/* + * 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.spark.resource + +import org.apache.spark.{SparkConf, SparkException, SparkFunSuite} +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Tests._ + +class ResourceProfileManagerSuite extends SparkFunSuite { + + override def beforeAll() { + try { + ResourceProfile.clearDefaultProfile() + } finally { + super.beforeAll() + } + } + + override def afterEach() { + try { + ResourceProfile.clearDefaultProfile() + } finally { + super.afterEach() + } + } + + test("ResourceProfileManager") { + val conf = new SparkConf().set(EXECUTOR_CORES, 4) + val rpmanager = new ResourceProfileManager(conf) + val defaultProf = rpmanager.defaultResourceProfile + assert(defaultProf.id === ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) + assert(defaultProf.executorResources.size === 2, + "Executor resources should contain cores and memory by default") + assert(defaultProf.executorResources(ResourceProfile.CORES).amount === 4, + s"Executor resources should have 4 cores") + } + + test("isSupported yarn no dynamic allocation") { + val conf = new SparkConf().setMaster("yarn").set(EXECUTOR_CORES, 4) + conf.set(RESOURCE_PROFILE_MANAGER_TESTING.key, "true") + val rpmanager = new ResourceProfileManager(conf) + // default profile should always work + val defaultProf = rpmanager.defaultResourceProfile + val rprof = new ResourceProfileBuilder() + val gpuExecReq = + new ExecutorResourceRequests().resource("gpu", 2, "someScript") + val immrprof = rprof.require(gpuExecReq).build + val error = intercept[SparkException] { + rpmanager.isSupported(immrprof) + }.getMessage() + + assert(error.contains("ResourceProfiles are only supported on YARN with dynamic allocation")) + } + + test("isSupported yarn with dynamic allocation") { + val conf = new SparkConf().setMaster("yarn").set(EXECUTOR_CORES, 4) + conf.set(DYN_ALLOCATION_ENABLED, true) + conf.set(RESOURCE_PROFILE_MANAGER_TESTING.key, "true") + val rpmanager = new ResourceProfileManager(conf) + // default profile should always work + val defaultProf = rpmanager.defaultResourceProfile + val rprof = new ResourceProfileBuilder() + val gpuExecReq = + new ExecutorResourceRequests().resource("gpu", 2, "someScript") + val immrprof = rprof.require(gpuExecReq).build + assert(rpmanager.isSupported(immrprof) == true) + } + + test("isSupported yarn with local mode") { + val conf = new SparkConf().setMaster("local").set(EXECUTOR_CORES, 4) + conf.set(RESOURCE_PROFILE_MANAGER_TESTING.key, "true") + val rpmanager = new ResourceProfileManager(conf) + // default profile should always work + val defaultProf = rpmanager.defaultResourceProfile + val rprof = new ResourceProfileBuilder() + val gpuExecReq = + new ExecutorResourceRequests().resource("gpu", 2, "someScript") + val immrprof = rprof.require(gpuExecReq).build + var error = intercept[SparkException] { + rpmanager.isSupported(immrprof) + }.getMessage() + + assert(error.contains("ResourceProfiles are only supported on YARN with dynamic allocation")) + } + + + +} diff --git a/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala b/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala index c0637eeeacaba..b2f2c3632e454 100644 --- a/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala +++ b/core/src/test/scala/org/apache/spark/resource/ResourceProfileSuite.scala @@ -18,18 +18,28 @@ package org.apache.spark.resource import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.internal.config.{EXECUTOR_CORES, EXECUTOR_MEMORY, EXECUTOR_MEMORY_OVERHEAD, SPARK_EXECUTOR_PREFIX} +import org.apache.spark.internal.config.{EXECUTOR_CORES, EXECUTOR_MEMORY, EXECUTOR_MEMORY_OVERHEAD} import org.apache.spark.internal.config.Python.PYSPARK_EXECUTOR_MEMORY +import org.apache.spark.resource.TestResourceIDs._ class ResourceProfileSuite extends SparkFunSuite { + override def beforeAll() { + try { + ResourceProfile.clearDefaultProfile() + } finally { + super.beforeAll() + } + } + override def afterEach() { try { - ResourceProfile.clearDefaultProfile + ResourceProfile.clearDefaultProfile() } finally { super.afterEach() } } + test("Default ResourceProfile") { val rprof = ResourceProfile.getOrCreateDefaultProfile(new SparkConf) assert(rprof.id === ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) @@ -59,18 +69,19 @@ class ResourceProfileSuite extends SparkFunSuite { conf.set(EXECUTOR_MEMORY_OVERHEAD.key, "1g") conf.set(EXECUTOR_MEMORY.key, "4g") conf.set(EXECUTOR_CORES.key, "4") - conf.set("spark.task.resource.gpu.amount", "1") - conf.set(s"$SPARK_EXECUTOR_PREFIX.resource.gpu.amount", "1") - conf.set(s"$SPARK_EXECUTOR_PREFIX.resource.gpu.discoveryScript", "nameOfScript") + conf.set(TASK_GPU_ID.amountConf, "1") + conf.set(EXECUTOR_GPU_ID.amountConf, "1") + conf.set(EXECUTOR_GPU_ID.discoveryScriptConf, "nameOfScript") val rprof = ResourceProfile.getOrCreateDefaultProfile(conf) assert(rprof.id === ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) val execResources = rprof.executorResources - assert(execResources.size === 5, - "Executor resources should contain cores, memory, and gpu " + execResources) + assert(execResources.size === 5, s"Executor resources should contain cores, pyspark " + + s"memory, memory overhead, memory, and gpu $execResources") assert(execResources.contains("gpu"), "Executor resources should have gpu") assert(rprof.executorResources(ResourceProfile.CORES).amount === 4, "Executor resources should have 4 core") - assert(rprof.getExecutorCores.get === 4, "Executor resources should have 4 core") + assert(rprof.getExecutorCores.get === 4, + "Executor resources should have 4 core") assert(rprof.executorResources(ResourceProfile.MEMORY).amount === 4096, "Executor resources should have 1024 memory") assert(rprof.executorResources(ResourceProfile.PYSPARK_MEM).amount == 2048, @@ -84,12 +95,60 @@ class ResourceProfileSuite extends SparkFunSuite { test("test default profile task gpus fractional") { val sparkConf = new SparkConf() - .set("spark.executor.resource.gpu.amount", "2") - .set("spark.task.resource.gpu.amount", "0.33") + .set(EXECUTOR_GPU_ID.amountConf, "2") + .set(TASK_GPU_ID.amountConf, "0.33") val immrprof = ResourceProfile.getOrCreateDefaultProfile(sparkConf) assert(immrprof.taskResources.get("gpu").get.amount == 0.33) } + test("maxTasksPerExecutor cpus") { + val sparkConf = new SparkConf() + .set(EXECUTOR_CORES, 1) + val rprof = new ResourceProfileBuilder() + val taskReq = new TaskResourceRequests().resource("gpu", 1) + val execReq = + new ExecutorResourceRequests().resource("gpu", 2, "myscript", "nvidia") + rprof.require(taskReq).require(execReq) + val immrprof = new ResourceProfile(rprof.executorResources, rprof.taskResources) + assert(immrprof.limitingResource(sparkConf) == "cpus") + assert(immrprof.maxTasksPerExecutor(sparkConf) == 1) + } + + test("maxTasksPerExecutor/limiting no executor cores") { + val sparkConf = new SparkConf().setMaster("spark://testing") + val rprof = new ResourceProfileBuilder() + val taskReq = new TaskResourceRequests().resource("gpu", 1) + val execReq = + new ExecutorResourceRequests().resource("gpu", 2, "myscript", "nvidia") + rprof.require(taskReq).require(execReq) + val immrprof = new ResourceProfile(rprof.executorResources, rprof.taskResources) + assert(immrprof.limitingResource(sparkConf) == "gpu") + assert(immrprof.maxTasksPerExecutor(sparkConf) == 2) + assert(immrprof.isCoresLimitKnown == false) + } + + test("maxTasksPerExecutor/limiting no other resource no executor cores") { + val sparkConf = new SparkConf().setMaster("spark://testing") + val immrprof = ResourceProfile.getOrCreateDefaultProfile(sparkConf) + assert(immrprof.limitingResource(sparkConf) == "") + assert(immrprof.maxTasksPerExecutor(sparkConf) == 1) + assert(immrprof.isCoresLimitKnown == false) + } + + test("maxTasksPerExecutor/limiting executor cores") { + val sparkConf = new SparkConf().setMaster("spark://testing").set(EXECUTOR_CORES, 2) + val rprof = new ResourceProfileBuilder() + val taskReq = new TaskResourceRequests().resource("gpu", 1) + val execReq = + new ExecutorResourceRequests().resource("gpu", 2, "myscript", "nvidia") + rprof.require(taskReq).require(execReq) + val immrprof = new ResourceProfile(rprof.executorResources, rprof.taskResources) + assert(immrprof.limitingResource(sparkConf) == ResourceProfile.CPUS) + assert(immrprof.maxTasksPerExecutor(sparkConf) == 2) + assert(immrprof.isCoresLimitKnown == true) + } + + test("Create ResourceProfile") { val rprof = new ResourceProfileBuilder() val taskReq = new TaskResourceRequests().resource("gpu", 1) diff --git a/core/src/test/scala/org/apache/spark/resource/ResourceUtilsSuite.scala b/core/src/test/scala/org/apache/spark/resource/ResourceUtilsSuite.scala index dffe9a02e9aa4..278a72a7192d8 100644 --- a/core/src/test/scala/org/apache/spark/resource/ResourceUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/resource/ResourceUtilsSuite.scala @@ -26,8 +26,10 @@ import org.json4s.{DefaultFormats, Extraction} import org.apache.spark.{LocalSparkContext, SparkConf, SparkException, SparkFunSuite} import org.apache.spark.TestUtils._ import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Tests._ import org.apache.spark.resource.ResourceUtils._ import org.apache.spark.resource.TestResourceIDs._ +import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.util.Utils class ResourceUtilsSuite extends SparkFunSuite @@ -165,6 +167,7 @@ class ResourceUtilsSuite extends SparkFunSuite val rpBuilder = new ResourceProfileBuilder() val ereqs = new ExecutorResourceRequests().resource(GPU, 2, gpuDiscovery) val treqs = new TaskResourceRequests().resource(GPU, 1) + val rp = rpBuilder.require(ereqs).require(treqs).build val resourcesFromBoth = getOrDiscoverAllResourcesForResourceProfile( Some(resourcesFile), SPARK_EXECUTOR_PREFIX, rp, conf) diff --git a/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala index fc8ac38479932..33594c0a50d14 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.scheduler import java.io.File +import scala.collection.mutable.ArrayBuffer import scala.util.Random import org.apache.spark._ @@ -52,6 +53,82 @@ class BarrierTaskContextSuite extends SparkFunSuite with LocalSparkContext { assert(times.max - times.min <= 1000) } + test("share messages with allGather() call") { + val conf = new SparkConf() + .setMaster("local-cluster[4, 1, 1024]") + .setAppName("test-cluster") + sc = new SparkContext(conf) + val rdd = sc.makeRDD(1 to 10, 4) + val rdd2 = rdd.barrier().mapPartitions { it => + val context = BarrierTaskContext.get() + // Sleep for a random time before global sync. + Thread.sleep(Random.nextInt(1000)) + // Pass partitionId message in + val message: String = context.partitionId().toString + val messages: ArrayBuffer[String] = context.allGather(message) + messages.toList.iterator + } + // Take a sorted list of all the partitionId messages + val messages = rdd2.collect().head + // All the task partitionIds are shared + for((x, i) <- messages.view.zipWithIndex) assert(x.toString == i.toString) + } + + test("throw exception if we attempt to synchronize with different blocking calls") { + val conf = new SparkConf() + .setMaster("local-cluster[4, 1, 1024]") + .setAppName("test-cluster") + sc = new SparkContext(conf) + val rdd = sc.makeRDD(1 to 10, 4) + val rdd2 = rdd.barrier().mapPartitions { it => + val context = BarrierTaskContext.get() + val partitionId = context.partitionId + if (partitionId == 0) { + context.barrier() + } else { + context.allGather(partitionId.toString) + } + Seq(null).iterator + } + val error = intercept[SparkException] { + rdd2.collect() + }.getMessage + assert( + error.contains("does not match the current synchronized requestMethod") || + error.contains("not properly killed") + ) + } + + test("successively sync with allGather and barrier") { + val conf = new SparkConf() + .setMaster("local-cluster[4, 1, 1024]") + .setAppName("test-cluster") + sc = new SparkContext(conf) + val rdd = sc.makeRDD(1 to 10, 4) + val rdd2 = rdd.barrier().mapPartitions { it => + val context = BarrierTaskContext.get() + // Sleep for a random time before global sync. + Thread.sleep(Random.nextInt(1000)) + context.barrier() + val time1 = System.currentTimeMillis() + // Sleep for a random time before global sync. + Thread.sleep(Random.nextInt(1000)) + // Pass partitionId message in + val message = context.partitionId().toString + val messages = context.allGather(message) + val time2 = System.currentTimeMillis() + Seq((time1, time2)).iterator + } + val times = rdd2.collect() + // All the tasks shall finish the first round of global sync within a short time slot. + val times1 = times.map(_._1) + assert(times1.max - times1.min <= 1000) + + // All the tasks shall finish the second round of global sync within a short time slot. + val times2 = times.map(_._2) + assert(times2.max - times2.min <= 1000) + } + test("support multiple barrier() call within a single task") { initLocalClusterSparkContext() val rdd = sc.makeRDD(1 to 10, 4) diff --git a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala index c063301673598..7666c6c7810cc 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala @@ -34,7 +34,7 @@ import org.apache.spark._ import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Network.RPC_MESSAGE_MAX_SIZE import org.apache.spark.rdd.RDD -import org.apache.spark.resource.{ResourceInformation, ResourceProfile} +import org.apache.spark.resource.{ExecutorResourceRequests, ResourceInformation, ResourceProfile, TaskResourceRequests} import org.apache.spark.resource.ResourceUtils._ import org.apache.spark.resource.TestResourceIDs._ import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcEnv} @@ -187,8 +187,6 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo } test("extra resources from executor") { - import TestUtils._ - val conf = new SparkConf() .set(EXECUTOR_CORES, 1) .set(SCHEDULER_REVIVE_INTERVAL.key, "1m") // don't let it auto revive during test @@ -200,6 +198,11 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo conf.set(EXECUTOR_GPU_ID.amountConf, "1") sc = new SparkContext(conf) + val execGpu = new ExecutorResourceRequests().cores(1).resource(GPU, 3) + val taskGpu = new TaskResourceRequests().cpus(1).resource(GPU, 1) + val rp = new ResourceProfile(execGpu.requests, taskGpu.requests) + sc.resourceProfileManager.addResourceProfile(rp) + assert(rp.id > ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) val backend = sc.schedulerBackend.asInstanceOf[TestCoarseGrainedSchedulerBackend] val mockEndpointRef = mock[RpcEndpointRef] val mockAddress = mock[RpcAddress] @@ -224,7 +227,7 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID)) backend.driverEndpoint.askSync[Boolean]( RegisterExecutor("3", mockEndpointRef, mockAddress.host, 1, Map.empty, Map.empty, resources, - 5)) + rp.id)) val frameSize = RpcUtils.maxMessageSizeBytes(sc.conf) val bytebuffer = java.nio.ByteBuffer.allocate(frameSize - 100) @@ -234,7 +237,7 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo assert(execResources(GPU).availableAddrs.sorted === Array("0", "1", "3")) var exec3ResourceProfileId = backend.getExecutorResourceProfileId("3") - assert(exec3ResourceProfileId === 5) + assert(exec3ResourceProfileId === rp.id) val taskResources = Map(GPU -> new ResourceInformation(GPU, Array("0"))) var taskDescs: Seq[Seq[TaskDescription]] = Seq(Seq(new TaskDescription(1, 0, "1", diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 101e60c73e9f8..e40b63fe13cb1 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -167,6 +167,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi } override def setDAGScheduler(dagScheduler: DAGScheduler) = {} override def defaultParallelism() = 2 + override def executorDecommission(executorId: String) = {} override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = {} override def workerRemoved(workerId: String, host: String, message: String): Unit = {} override def applicationAttemptId(): Option[String] = None @@ -707,6 +708,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], blockManagerId: BlockManagerId, executorUpdates: Map[(Int, Int), ExecutorMetrics]): Boolean = true + override def executorDecommission(executorId: String): Unit = {} override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = {} override def workerRemoved(workerId: String, host: String, message: String): Unit = {} override def applicationAttemptId(): Option[String] = None diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index 286924001e920..61ea21fa86c5a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -38,6 +38,7 @@ import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.internal.Logging import org.apache.spark.io._ import org.apache.spark.metrics.{ExecutorMetricType, MetricsSystem} +import org.apache.spark.resource.ResourceProfile import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.util.{JsonProtocol, Utils} @@ -438,12 +439,14 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit private def createStageSubmittedEvent(stageId: Int) = { SparkListenerStageSubmitted(new StageInfo(stageId, 0, stageId.toString, 0, - Seq.empty, Seq.empty, "details")) + Seq.empty, Seq.empty, "details", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID)) } private def createStageCompletedEvent(stageId: Int) = { SparkListenerStageCompleted(new StageInfo(stageId, 0, stageId.toString, 0, - Seq.empty, Seq.empty, "details")) + Seq.empty, Seq.empty, "details", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID)) } private def createExecutorAddedEvent(executorId: Int) = { diff --git a/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala index 4e71ec1ea7b37..9f593e0039adc 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala @@ -89,6 +89,7 @@ private class DummyTaskScheduler extends TaskScheduler { override def notifyPartitionCompletion(stageId: Int, partitionId: Int): Unit = {} override def setDAGScheduler(dagScheduler: DAGScheduler): Unit = {} override def defaultParallelism(): Int = 2 + override def executorDecommission(executorId: String): Unit = {} override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = {} override def workerRemoved(workerId: String, host: String, message: String): Unit = {} override def applicationAttemptId(): Option[String] = None diff --git a/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionSuite.scala new file mode 100644 index 0000000000000..15733b0d932ec --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionSuite.scala @@ -0,0 +1,84 @@ +/* + * 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.spark.scheduler + +import java.util.concurrent.Semaphore + +import scala.concurrent.TimeoutException +import scala.concurrent.duration._ + +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkException, SparkFunSuite} +import org.apache.spark.internal.config +import org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend +import org.apache.spark.util.{RpcUtils, SerializableBuffer, ThreadUtils} + +class WorkerDecommissionSuite extends SparkFunSuite with LocalSparkContext { + + override def beforeEach(): Unit = { + val conf = new SparkConf().setAppName("test").setMaster("local") + .set(config.Worker.WORKER_DECOMMISSION_ENABLED, true) + + sc = new SparkContext("local-cluster[2, 1, 1024]", "test", conf) + } + + test("verify task with no decommissioning works as expected") { + val input = sc.parallelize(1 to 10) + input.count() + val sleepyRdd = input.mapPartitions{ x => + Thread.sleep(100) + x + } + assert(sleepyRdd.count() === 10) + } + + test("verify a task with all workers decommissioned succeeds") { + val input = sc.parallelize(1 to 10) + // Do a count to wait for the executors to be registered. + input.count() + val sleepyRdd = input.mapPartitions{ x => + Thread.sleep(50) + x + } + // Listen for the job + val sem = new Semaphore(0) + sc.addSparkListener(new SparkListener { + override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = { + sem.release() + } + }) + // Start the task. + val asyncCount = sleepyRdd.countAsync() + // Wait for the job to have started + sem.acquire(1) + // Decommission all the executors, this should not halt the current task. + // decom.sh message passing is tested manually. + val sched = sc.schedulerBackend.asInstanceOf[StandaloneSchedulerBackend] + val execs = sched.getExecutorIds() + execs.foreach(execId => sched.decommissionExecutor(execId)) + val asyncCountResult = ThreadUtils.awaitResult(asyncCount, 2.seconds) + assert(asyncCountResult === 10) + // Try and launch task after decommissioning, this should fail + val postDecommissioned = input.map(x => x) + val postDecomAsyncCount = postDecommissioned.countAsync() + val thrown = intercept[java.util.concurrent.TimeoutException]{ + val result = ThreadUtils.awaitResult(postDecomAsyncCount, 2.seconds) + } + assert(postDecomAsyncCount.isCompleted === false, + "After exec decommission new task could not launch") + } +} diff --git a/core/src/test/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitorSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitorSuite.scala index 615389ae5c2d4..3596a9ebb1f5a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitorSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitorSuite.scala @@ -28,6 +28,7 @@ import org.apache.spark._ import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.internal.config._ import org.apache.spark.resource.ResourceProfile.{DEFAULT_RESOURCE_PROFILE_ID, UNKNOWN_RESOURCE_PROFILE_ID} +import org.apache.spark.resource.ResourceProfile import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.storage._ @@ -255,25 +256,28 @@ class ExecutorMonitorSuite extends SparkFunSuite { test("track executors pending for removal") { knownExecs ++= Set("1", "2", "3") + val execInfoRp1 = new ExecutorInfo("host1", 1, Map.empty, + Map.empty, Map.empty, 1) + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", execInfo)) monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "2", execInfo)) - monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "3", execInfo)) + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "3", execInfoRp1)) clock.setTime(idleDeadline) - assert(monitor.timedOutExecutors().toSet === Set("1", "2", "3")) + assert(monitor.timedOutExecutors().toSet === Set(("1", 0), ("2", 0), ("3", 1))) assert(monitor.pendingRemovalCount === 0) // Notify that only a subset of executors was killed, to mimic the case where the scheduler // refuses to kill an executor that is busy for whatever reason the monitor hasn't detected yet. monitor.executorsKilled(Seq("1")) - assert(monitor.timedOutExecutors().toSet === Set("2", "3")) + assert(monitor.timedOutExecutors().toSet === Set(("2", 0), ("3", 1))) assert(monitor.pendingRemovalCount === 1) // Check the timed out executors again so that we're sure they're still timed out when no // events happen. This ensures that the monitor doesn't lose track of them. - assert(monitor.timedOutExecutors().toSet === Set("2", "3")) + assert(monitor.timedOutExecutors().toSet === Set(("2", 0), ("3", 1))) monitor.onTaskStart(SparkListenerTaskStart(1, 1, taskInfo("2", 1))) - assert(monitor.timedOutExecutors().toSet === Set("3")) + assert(monitor.timedOutExecutors().toSet === Set(("3", 1))) monitor.executorsKilled(Seq("3")) assert(monitor.pendingRemovalCount === 2) @@ -282,7 +286,7 @@ class ExecutorMonitorSuite extends SparkFunSuite { new ExecutorMetrics, null)) assert(monitor.timedOutExecutors().isEmpty) clock.advance(idleDeadline) - assert(monitor.timedOutExecutors().toSet === Set("2")) + assert(monitor.timedOutExecutors().toSet === Set(("2", 0))) } test("shuffle block tracking") { @@ -435,7 +439,8 @@ class ExecutorMonitorSuite extends SparkFunSuite { private def stageInfo(id: Int, shuffleId: Int = -1): StageInfo = { new StageInfo(id, 0, s"stage$id", 1, Nil, Nil, "", - shuffleDepId = if (shuffleId >= 0) Some(shuffleId) else None) + shuffleDepId = if (shuffleId >= 0) Some(shuffleId) else None, + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) } private def taskInfo( diff --git a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala index 255f91866ef58..24eb1685f577a 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala @@ -30,6 +30,7 @@ import org.apache.spark._ import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.internal.config.Status._ import org.apache.spark.metrics.ExecutorMetricType +import org.apache.spark.resource.ResourceProfile import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster._ import org.apache.spark.status.ListenerEventsTestHelper._ @@ -151,8 +152,10 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { // Start a job with 2 stages / 4 tasks each time += 1 val stages = Seq( - new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1"), - new StageInfo(2, 0, "stage2", 4, Nil, Seq(1), "details2")) + new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID), + new StageInfo(2, 0, "stage2", 4, Nil, Seq(1), "details2", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID)) val jobProps = new Properties() jobProps.setProperty(SparkContext.SPARK_JOB_DESCRIPTION, "jobDescription") @@ -524,7 +527,8 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { // - Re-submit stage 2, all tasks, and succeed them and the stage. val oldS2 = stages.last val newS2 = new StageInfo(oldS2.stageId, oldS2.attemptNumber + 1, oldS2.name, oldS2.numTasks, - oldS2.rddInfos, oldS2.parentIds, oldS2.details, oldS2.taskMetrics) + oldS2.rddInfos, oldS2.parentIds, oldS2.details, oldS2.taskMetrics, + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) time += 1 newS2.submissionTime = Some(time) @@ -575,8 +579,10 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { // change the stats of the already finished job. time += 1 val j2Stages = Seq( - new StageInfo(3, 0, "stage1", 4, Nil, Nil, "details1"), - new StageInfo(4, 0, "stage2", 4, Nil, Seq(3), "details2")) + new StageInfo(3, 0, "stage1", 4, Nil, Nil, "details1", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID), + new StageInfo(4, 0, "stage2", 4, Nil, Seq(3), "details2", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID)) j2Stages.last.submissionTime = Some(time) listener.onJobStart(SparkListenerJobStart(2, time, j2Stages, null)) assert(store.count(classOf[JobDataWrapper]) === 2) @@ -703,7 +709,8 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { // Submit a stage for the first RDD before it's marked for caching, to make sure later // the listener picks up the correct storage level. val rdd1Info = new RDDInfo(rdd1b1.rddId, "rdd1", 2, StorageLevel.NONE, false, Nil) - val stage0 = new StageInfo(0, 0, "stage0", 4, Seq(rdd1Info), Nil, "details0") + val stage0 = new StageInfo(0, 0, "stage0", 4, Seq(rdd1Info), Nil, "details0", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) listener.onStageSubmitted(SparkListenerStageSubmitted(stage0, new Properties())) listener.onStageCompleted(SparkListenerStageCompleted(stage0)) assert(store.count(classOf[RDDStorageInfoWrapper]) === 0) @@ -711,7 +718,8 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { // Submit a stage and make sure the RDDs are recorded. rdd1Info.storageLevel = level val rdd2Info = new RDDInfo(rdd2b1.rddId, "rdd2", 1, level, false, Nil) - val stage = new StageInfo(1, 0, "stage1", 4, Seq(rdd1Info, rdd2Info), Nil, "details1") + val stage = new StageInfo(1, 0, "stage1", 4, Seq(rdd1Info, rdd2Info), Nil, "details1", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) listener.onStageSubmitted(SparkListenerStageSubmitted(stage, new Properties())) check[RDDStorageInfoWrapper](rdd1b1.rddId) { wrapper => @@ -1018,9 +1026,12 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { // data is not deleted. time += 1 val stages = Seq( - new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1"), - new StageInfo(2, 0, "stage2", 4, Nil, Nil, "details2"), - new StageInfo(3, 0, "stage3", 4, Nil, Nil, "details3")) + new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID), + new StageInfo(2, 0, "stage2", 4, Nil, Nil, "details2", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID), + new StageInfo(3, 0, "stage3", 4, Nil, Nil, "details3", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID)) // Graph data is generated by the job start event, so fire it. listener.onJobStart(SparkListenerJobStart(4, time, stages, null)) @@ -1068,7 +1079,8 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { } assert(store.count(classOf[CachedQuantile], "stage", key(dropped)) === 0) - val attempt2 = new StageInfo(3, 1, "stage3", 4, Nil, Nil, "details3") + val attempt2 = new StageInfo(3, 1, "stage3", 4, Nil, Nil, "details3", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) time += 1 attempt2.submissionTime = Some(time) listener.onStageSubmitted(SparkListenerStageSubmitted(attempt2, new Properties())) @@ -1139,9 +1151,12 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { val testConf = conf.clone().set(MAX_RETAINED_STAGES, 2) val listener = new AppStatusListener(store, testConf, true) - val stage1 = new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1") - val stage2 = new StageInfo(2, 0, "stage2", 4, Nil, Nil, "details2") - val stage3 = new StageInfo(3, 0, "stage3", 4, Nil, Nil, "details3") + val stage1 = new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) + val stage2 = new StageInfo(2, 0, "stage2", 4, Nil, Nil, "details2", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) + val stage3 = new StageInfo(3, 0, "stage3", 4, Nil, Nil, "details3", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) // Start stage 1 and stage 2 time += 1 @@ -1172,8 +1187,10 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { val testConf = conf.clone().set(MAX_RETAINED_STAGES, 2) val listener = new AppStatusListener(store, testConf, true) - val stage1 = new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1") - val stage2 = new StageInfo(2, 0, "stage2", 4, Nil, Nil, "details2") + val stage1 = new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) + val stage2 = new StageInfo(2, 0, "stage2", 4, Nil, Nil, "details2", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) // Sart job 1 time += 1 @@ -1193,7 +1210,8 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { listener.onJobEnd(SparkListenerJobEnd(1, time, JobSucceeded)) // Submit stage 3 and verify stage 2 is evicted - val stage3 = new StageInfo(3, 0, "stage3", 4, Nil, Nil, "details3") + val stage3 = new StageInfo(3, 0, "stage3", 4, Nil, Nil, "details3", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) time += 1 stage3.submissionTime = Some(time) listener.onStageSubmitted(SparkListenerStageSubmitted(stage3, new Properties())) @@ -1208,7 +1226,8 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { val testConf = conf.clone().set(MAX_RETAINED_TASKS_PER_STAGE, 2) val listener = new AppStatusListener(store, testConf, true) - val stage1 = new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1") + val stage1 = new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) stage1.submissionTime = Some(time) listener.onStageSubmitted(SparkListenerStageSubmitted(stage1, new Properties())) @@ -1243,9 +1262,12 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { val listener = new AppStatusListener(store, testConf, true) val appStore = new AppStatusStore(store) - val stage1 = new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1") - val stage2 = new StageInfo(2, 0, "stage2", 4, Nil, Nil, "details2") - val stage3 = new StageInfo(3, 0, "stage3", 4, Nil, Nil, "details3") + val stage1 = new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) + val stage2 = new StageInfo(2, 0, "stage2", 4, Nil, Nil, "details2", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) + val stage3 = new StageInfo(3, 0, "stage3", 4, Nil, Nil, "details3", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) time += 1 stage1.submissionTime = Some(time) @@ -1274,8 +1296,10 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { test("SPARK-24415: update metrics for tasks that finish late") { val listener = new AppStatusListener(store, conf, true) - val stage1 = new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1") - val stage2 = new StageInfo(2, 0, "stage2", 4, Nil, Nil, "details2") + val stage1 = new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) + val stage2 = new StageInfo(2, 0, "stage2", 4, Nil, Nil, "details2", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) // Start job listener.onJobStart(SparkListenerJobStart(1, time, Seq(stage1, stage2), null)) @@ -1340,7 +1364,8 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { listener.onExecutorAdded(createExecutorAddedEvent(1)) listener.onExecutorAdded(createExecutorAddedEvent(2)) - val stage = new StageInfo(1, 0, "stage", 4, Nil, Nil, "details") + val stage = new StageInfo(1, 0, "stage", 4, Nil, Nil, "details", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) listener.onJobStart(SparkListenerJobStart(1, time, Seq(stage), null)) listener.onStageSubmitted(SparkListenerStageSubmitted(stage, new Properties())) @@ -1577,7 +1602,8 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { // Submit a stage and make sure the RDDs are recorded. val rdd1Info = new RDDInfo(rdd1b1.rddId, "rdd1", 2, level, false, Nil) - val stage = new StageInfo(1, 0, "stage1", 4, Seq(rdd1Info), Nil, "details1") + val stage = new StageInfo(1, 0, "stage1", 4, Seq(rdd1Info), Nil, "details1", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) listener.onStageSubmitted(SparkListenerStageSubmitted(stage, new Properties())) // Add partition 1 replicated on two block managers. diff --git a/core/src/test/scala/org/apache/spark/status/ListenerEventsTestHelper.scala b/core/src/test/scala/org/apache/spark/status/ListenerEventsTestHelper.scala index 4b3fbacc47f9c..99c0d9593ccae 100644 --- a/core/src/test/scala/org/apache/spark/status/ListenerEventsTestHelper.scala +++ b/core/src/test/scala/org/apache/spark/status/ListenerEventsTestHelper.scala @@ -23,6 +23,7 @@ import scala.collection.immutable.Map import org.apache.spark.{AccumulatorSuite, SparkContext, Success, TaskState} import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} +import org.apache.spark.resource.ResourceProfile import org.apache.spark.scheduler.{SparkListener, SparkListenerExecutorAdded, SparkListenerExecutorMetricsUpdate, SparkListenerExecutorRemoved, SparkListenerJobStart, SparkListenerStageCompleted, SparkListenerStageSubmitted, SparkListenerTaskEnd, SparkListenerTaskStart, StageInfo, TaskInfo, TaskLocality} import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.storage.{RDDInfo, StorageLevel} @@ -61,7 +62,8 @@ object ListenerEventsTestHelper { } def createStage(id: Int, rdds: Seq[RDDInfo], parentIds: Seq[Int]): StageInfo = { - new StageInfo(id, 0, s"stage${id}", 4, rdds, parentIds, s"details${id}") + new StageInfo(id, 0, s"stage${id}", 4, rdds, parentIds, s"details${id}", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) } def createStage(rdds: Seq[RDDInfo], parentIds: Seq[Int]): StageInfo = { @@ -96,13 +98,15 @@ object ListenerEventsTestHelper { /** Create a stage submitted event for the specified stage Id. */ def createStageSubmittedEvent(stageId: Int): SparkListenerStageSubmitted = { SparkListenerStageSubmitted(new StageInfo(stageId, 0, stageId.toString, 0, - Seq.empty, Seq.empty, "details")) + Seq.empty, Seq.empty, "details", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID)) } /** Create a stage completed event for the specified stage Id. */ def createStageCompletedEvent(stageId: Int): SparkListenerStageCompleted = { SparkListenerStageCompleted(new StageInfo(stageId, 0, stageId.toString, 0, - Seq.empty, Seq.empty, "details")) + Seq.empty, Seq.empty, "details", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID)) } def createExecutorAddedEvent(executorId: Int): SparkListenerExecutorAdded = { diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala index 59ace850d0bd2..660bfcfc48267 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala @@ -24,7 +24,8 @@ import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration._ import scala.language.implicitConversions -import org.mockito.Mockito.{mock, when} +import org.mockito.ArgumentMatchers.any +import org.mockito.Mockito.{doAnswer, mock, spy, when} import org.scalatest.{BeforeAndAfter, Matchers} import org.scalatest.concurrent.Eventually._ @@ -69,11 +70,12 @@ trait BlockManagerReplicationBehavior extends SparkFunSuite protected def makeBlockManager( maxMem: Long, - name: String = SparkContext.DRIVER_IDENTIFIER): BlockManager = { + name: String = SparkContext.DRIVER_IDENTIFIER, + memoryManager: Option[UnifiedMemoryManager] = None): BlockManager = { conf.set(TEST_MEMORY, maxMem) conf.set(MEMORY_OFFHEAP_SIZE, maxMem) val transfer = new NettyBlockTransferService(conf, securityMgr, "localhost", "localhost", 0, 1) - val memManager = UnifiedMemoryManager(conf, numCores = 1) + val memManager = memoryManager.getOrElse(UnifiedMemoryManager(conf, numCores = 1)) val serializerManager = new SerializerManager(serializer, conf) val store = new BlockManager(name, rpcEnv, master, serializerManager, conf, memManager, mapOutputTracker, shuffleManager, transfer, securityMgr, None) @@ -255,6 +257,43 @@ trait BlockManagerReplicationBehavior extends SparkFunSuite } } + Seq(false, true).foreach { stream => + test(s"test block replication failures when block is received " + + s"by remote block manager but putBlock fails (stream = $stream)") { + // Retry replication logic for 1 failure + conf.set(STORAGE_MAX_REPLICATION_FAILURE, 1) + // Custom block replication policy which prioritizes BlockManagers as per hostnames + conf.set(STORAGE_REPLICATION_POLICY, classOf[SortOnHostNameBlockReplicationPolicy].getName) + // To use upload block stream flow, set maxRemoteBlockSizeFetchToMem to 0 + val maxRemoteBlockSizeFetchToMem = if (stream) 0 else Int.MaxValue - 512 + conf.set(MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM, maxRemoteBlockSizeFetchToMem.toLong) + + // Create 2 normal block manager + val store1 = makeBlockManager(10000, "host-1") + val store3 = makeBlockManager(10000, "host-3") + + // create 1 faulty block manager by injecting faulty memory manager + val memManager = UnifiedMemoryManager(conf, numCores = 1) + val mockedMemoryManager = spy(memManager) + doAnswer(_ => false).when(mockedMemoryManager).acquireStorageMemory(any(), any(), any()) + val store2 = makeBlockManager(10000, "host-2", Some(mockedMemoryManager)) + + assert(master.getPeers(store1.blockManagerId).toSet === + Set(store2.blockManagerId, store3.blockManagerId)) + + val blockId = "blockId" + val message = new Array[Byte](1000) + + // Replication will be tried by store1 in this order: store2, store3 + // store2 is faulty block manager, so it won't be able to put block + // Then store1 will try to replicate block on store3 + store1.putSingle(blockId, message, StorageLevel.MEMORY_ONLY_SER_2) + + val blockLocations = master.getLocations(blockId).toSet + assert(blockLocations === Set(store1.blockManagerId, store3.blockManagerId)) + } + } + test("block replication - addition and deletion of block managers") { val blockSize = 1000 val storeSize = 10000 @@ -509,3 +548,17 @@ class BlockManagerBasicStrategyReplicationSuite extends BlockManagerReplicationB classOf[DummyTopologyMapper].getName) } +// BlockReplicationPolicy to prioritize BlockManagers based on hostnames +// Examples - for BM-x(host-2), BM-y(host-1), BM-z(host-3), it will prioritize them as +// BM-y(host-1), BM-x(host-2), BM-z(host-3) +class SortOnHostNameBlockReplicationPolicy + extends BlockReplicationPolicy { + override def prioritize( + blockManagerId: BlockManagerId, + peers: Seq[BlockManagerId], + peersReplicatedTo: mutable.HashSet[BlockManagerId], + blockId: BlockId, + numReplicas: Int): List[BlockManagerId] = { + peers.sortBy(_.host).toList + } +} diff --git a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala index bd18e9e628da8..7711934cbe8a6 100644 --- a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala @@ -27,6 +27,7 @@ import org.mockito.Mockito.{mock, when, RETURNS_SMART_NULLS} import org.apache.spark._ import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.internal.config.Status._ +import org.apache.spark.resource.ResourceProfile import org.apache.spark.scheduler._ import org.apache.spark.status.AppStatusStore import org.apache.spark.status.api.v1.{AccumulableInfo => UIAccumulableInfo, StageData, StageStatus} @@ -131,7 +132,8 @@ class StagePageSuite extends SparkFunSuite with LocalSparkContext { val page = new StagePage(tab, statusStore) // Simulate a stage in job progress listener - val stageInfo = new StageInfo(0, 0, "dummy", 1, Seq.empty, Seq.empty, "details") + val stageInfo = new StageInfo(0, 0, "dummy", 1, Seq.empty, Seq.empty, "details", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) // Simulate two tasks to test PEAK_EXECUTION_MEMORY correctness (1 to 2).foreach { taskId => diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index a2a4b3aa974fc..eb7f3079bee36 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -32,8 +32,7 @@ import org.apache.spark._ import org.apache.spark.executor._ import org.apache.spark.metrics.ExecutorMetricType import org.apache.spark.rdd.RDDOperationScope -import org.apache.spark.resource.ResourceInformation -import org.apache.spark.resource.ResourceUtils +import org.apache.spark.resource.{ResourceInformation, ResourceProfile, ResourceUtils} import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.shuffle.MetadataFetchFailedException @@ -341,7 +340,8 @@ class JsonProtocolSuite extends SparkFunSuite { val stageIds = Seq[Int](1, 2, 3, 4) val stageInfos = stageIds.map(x => makeStageInfo(x, x * 200, x * 300, x * 400L, x * 500L)) val dummyStageInfos = - stageIds.map(id => new StageInfo(id, 0, "unknown", 0, Seq.empty, Seq.empty, "unknown")) + stageIds.map(id => new StageInfo(id, 0, "unknown", 0, Seq.empty, Seq.empty, "unknown", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID)) val jobStart = SparkListenerJobStart(10, jobSubmissionTime, stageInfos, properties) val oldEvent = JsonProtocol.jobStartToJson(jobStart).removeField({_._1 == "Stage Infos"}) val expectedJobStart = @@ -383,9 +383,11 @@ class JsonProtocolSuite extends SparkFunSuite { test("StageInfo backward compatibility (parent IDs)") { // Prior to Spark 1.4.0, StageInfo did not have the "Parent IDs" property - val stageInfo = new StageInfo(1, 1, "me-stage", 1, Seq.empty, Seq(1, 2, 3), "details") + val stageInfo = new StageInfo(1, 1, "me-stage", 1, Seq.empty, Seq(1, 2, 3), "details", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) val oldStageInfo = JsonProtocol.stageInfoToJson(stageInfo).removeField({ _._1 == "Parent IDs"}) - val expectedStageInfo = new StageInfo(1, 1, "me-stage", 1, Seq.empty, Seq.empty, "details") + val expectedStageInfo = new StageInfo(1, 1, "me-stage", 1, Seq.empty, Seq.empty, "details", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) assertEquals(expectedStageInfo, JsonProtocol.stageInfoFromJson(oldStageInfo)) } @@ -481,6 +483,28 @@ class JsonProtocolSuite extends SparkFunSuite { testAccumValue(Some("anything"), blocks, JString(blocks.toString)) testAccumValue(Some("anything"), 123, JString("123")) } + + test("SPARK-30936: forwards compatibility - ignore unknown fields") { + val expected = TestListenerEvent("foo", 123) + val unknownFieldsJson = + """{ + | "Event" : "org.apache.spark.util.TestListenerEvent", + | "foo" : "foo", + | "bar" : 123, + | "unknown" : "unknown" + |}""".stripMargin + assert(JsonProtocol.sparkEventFromJson(parse(unknownFieldsJson)) === expected) + } + + test("SPARK-30936: backwards compatibility - set default values for missing fields") { + val expected = TestListenerEvent("foo", 0) + val unknownFieldsJson = + """{ + | "Event" : "org.apache.spark.util.TestListenerEvent", + | "foo" : "foo" + |}""".stripMargin + assert(JsonProtocol.sparkEventFromJson(parse(unknownFieldsJson)) === expected) + } } @@ -873,7 +897,8 @@ private[spark] object JsonProtocolSuite extends Assertions { private def makeStageInfo(a: Int, b: Int, c: Int, d: Long, e: Long) = { val rddInfos = (0 until a % 5).map { i => makeRddInfo(a + i, b + i, c + i, d + i, e + i) } - val stageInfo = new StageInfo(a, 0, "greetings", b, rddInfos, Seq(100, 200, 300), "details") + val stageInfo = new StageInfo(a, 0, "greetings", b, rddInfos, Seq(100, 200, 300), "details", + resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) val (acc1, acc2) = (makeAccumulableInfo(1), makeAccumulableInfo(2)) stageInfo.accumulables(acc1.id) = acc1 stageInfo.accumulables(acc2.id) = acc2 @@ -2310,3 +2335,5 @@ private[spark] object JsonProtocolSuite extends Assertions { |} """.stripMargin } + +case class TestListenerEvent(foo: String, bar: Int) extends SparkListenerEvent diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index 8f8902e497d49..f5e438b0f1a52 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -1243,6 +1243,10 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { intercept[IllegalArgumentException] { Utils.checkAndGetK8sMasterUrl("k8s://foo://host:port") } + + intercept[IllegalArgumentException] { + Utils.checkAndGetK8sMasterUrl("k8s:///https://host:port") + } } test("stringHalfWidth") { diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 index 534ac39e0c46e..247798547f79a 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-1.2 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-1.2 @@ -174,7 +174,7 @@ parquet-hadoop-bundle/1.6.0//parquet-hadoop-bundle-1.6.0.jar parquet-hadoop/1.10.1//parquet-hadoop-1.10.1.jar parquet-jackson/1.10.1//parquet-jackson-1.10.1.jar protobuf-java/2.5.0//protobuf-java-2.5.0.jar -py4j/0.10.8.1//py4j-0.10.8.1.jar +py4j/0.10.9//py4j-0.10.9.jar pyrolite/4.30//pyrolite-4.30.jar scala-collection-compat_2.12/2.1.1//scala-collection-compat_2.12-2.1.1.jar scala-compiler/2.12.10//scala-compiler-2.12.10.jar diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 index 42bdf112efccb..60883a58957db 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 @@ -87,7 +87,6 @@ hive-jdbc/2.3.6//hive-jdbc-2.3.6.jar hive-llap-common/2.3.6//hive-llap-common-2.3.6.jar hive-metastore/2.3.6//hive-metastore-2.3.6.jar hive-serde/2.3.6//hive-serde-2.3.6.jar -hive-service-rpc/2.3.6//hive-service-rpc-2.3.6.jar hive-shims-0.23/2.3.6//hive-shims-0.23-2.3.6.jar hive-shims-common/2.3.6//hive-shims-common-2.3.6.jar hive-shims-scheduler/2.3.6//hive-shims-scheduler-2.3.6.jar @@ -189,7 +188,7 @@ parquet-format/2.4.0//parquet-format-2.4.0.jar parquet-hadoop/1.10.1//parquet-hadoop-1.10.1.jar parquet-jackson/1.10.1//parquet-jackson-1.10.1.jar protobuf-java/2.5.0//protobuf-java-2.5.0.jar -py4j/0.10.8.1//py4j-0.10.8.1.jar +py4j/0.10.9//py4j-0.10.9.jar pyrolite/4.30//pyrolite-4.30.jar scala-collection-compat_2.12/2.1.1//scala-collection-compat_2.12-2.1.1.jar scala-compiler/2.12.10//scala-compiler-2.12.10.jar diff --git a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 index 6006fa4b43f42..944415e06f3ba 100644 --- a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 @@ -86,7 +86,6 @@ hive-jdbc/2.3.6//hive-jdbc-2.3.6.jar hive-llap-common/2.3.6//hive-llap-common-2.3.6.jar hive-metastore/2.3.6//hive-metastore-2.3.6.jar hive-serde/2.3.6//hive-serde-2.3.6.jar -hive-service-rpc/2.3.6//hive-service-rpc-2.3.6.jar hive-shims-0.23/2.3.6//hive-shims-0.23-2.3.6.jar hive-shims-common/2.3.6//hive-shims-common-2.3.6.jar hive-shims-scheduler/2.3.6//hive-shims-scheduler-2.3.6.jar @@ -204,7 +203,7 @@ parquet-format/2.4.0//parquet-format-2.4.0.jar parquet-hadoop/1.10.1//parquet-hadoop-1.10.1.jar parquet-jackson/1.10.1//parquet-jackson-1.10.1.jar protobuf-java/2.5.0//protobuf-java-2.5.0.jar -py4j/0.10.8.1//py4j-0.10.8.1.jar +py4j/0.10.9//py4j-0.10.9.jar pyrolite/4.30//pyrolite-4.30.jar re2j/1.1//re2j-1.1.jar scala-collection-compat_2.12/2.1.1//scala-collection-compat_2.12-2.1.1.jar diff --git a/dev/make-distribution.sh b/dev/make-distribution.sh index 0b30eec76bb53..9e767ce5a3daa 100755 --- a/dev/make-distribution.sh +++ b/dev/make-distribution.sh @@ -39,6 +39,7 @@ NAME=none MVN="$SPARK_HOME/build/mvn" function exit_with_usage { + set +x echo "make-distribution.sh - tool for making binary distributions of Spark" echo "" echo "usage:" diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index 40f2ca288d694..391e4bbe1b1f0 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -364,7 +364,6 @@ def __hash__(self): "pyspark.sql.avro.functions", "pyspark.sql.pandas.conversion", "pyspark.sql.pandas.map_ops", - "pyspark.sql.pandas.functions", "pyspark.sql.pandas.group_ops", "pyspark.sql.pandas.types", "pyspark.sql.pandas.serializers", diff --git a/docs/_config.yml b/docs/_config.yml index a888620139207..f82394ed63694 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,8 +14,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 3.0.0-SNAPSHOT -SPARK_VERSION_SHORT: 3.0.0 +SPARK_VERSION: 3.1.0-SNAPSHOT +SPARK_VERSION_SHORT: 3.1.0 SCALA_BINARY_VERSION: "2.12" SCALA_VERSION: "2.12.10" MESOS_VERSION: 1.0.0 diff --git a/docs/_data/menu-sql.yaml b/docs/_data/menu-sql.yaml index 241ec399d7bd5..38a5cf61245a6 100644 --- a/docs/_data/menu-sql.yaml +++ b/docs/_data/menu-sql.yaml @@ -80,6 +80,15 @@ url: sql-ref-null-semantics.html - text: NaN Semantics url: sql-ref-nan-semantics.html + - text: ANSI Compliance + url: sql-ref-ansi-compliance.html + subitems: + - text: Arithmetic Operations + url: sql-ref-ansi-compliance.html#arithmetic-operations + - text: Type Conversion + url: sql-ref-ansi-compliance.html#type-conversion + - text: SQL Keywords + url: sql-ref-ansi-compliance.html#sql-keywords - text: SQL Syntax url: sql-ref-syntax.html subitems: @@ -148,12 +157,12 @@ - text: Auxiliary Statements url: sql-ref-syntax-aux.html subitems: - - text: Analyze statement + - text: ANALYZE url: sql-ref-syntax-aux-analyze.html subitems: - text: ANALYZE TABLE url: sql-ref-syntax-aux-analyze-table.html - - text: Caching statements + - text: CACHE url: sql-ref-syntax-aux-cache.html subitems: - text: CACHE TABLE @@ -166,7 +175,7 @@ url: sql-ref-syntax-aux-refresh-table.html - text: REFRESH url: sql-ref-syntax-aux-cache-refresh.md - - text: Describe Commands + - text: DESCRIBE url: sql-ref-syntax-aux-describe.html subitems: - text: DESCRIBE DATABASE @@ -177,7 +186,7 @@ url: sql-ref-syntax-aux-describe-function.html - text: DESCRIBE QUERY url: sql-ref-syntax-aux-describe-query.html - - text: Show commands + - text: SHOW url: sql-ref-syntax-aux-show.html subitems: - text: SHOW COLUMNS @@ -196,14 +205,14 @@ url: sql-ref-syntax-aux-show-partitions.html - text: SHOW CREATE TABLE url: sql-ref-syntax-aux-show-create-table.html - - text: Configuration Management Commands + - text: CONFIGURATION MANAGEMENT url: sql-ref-syntax-aux-conf-mgmt.html subitems: - text: SET url: sql-ref-syntax-aux-conf-mgmt-set.html - text: RESET url: sql-ref-syntax-aux-conf-mgmt-reset.html - - text: Resource Management Commands + - text: RESOURCE MANAGEMENT url: sql-ref-syntax-aux-resource-mgmt.html subitems: - text: ADD FILE @@ -214,5 +223,3 @@ url: sql-ref-syntax-aux-resource-mgmt-list-file.html - text: LIST JAR url: sql-ref-syntax-aux-resource-mgmt-list-jar.html - - text: Arithmetic operations - url: sql-ref-arithmetic-ops.html diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index d5fb18bfb06c0..d05ac6bbe129d 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -82,7 +82,7 @@