From 999ec137a97844abbbd483dd98c7ded2f8ff356c Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Fri, 1 Dec 2017 02:28:24 +0800 Subject: [PATCH 01/37] [SPARK-22570][SQL] Avoid to create a lot of global variables by using a local variable with allocation of an object in generated code ## What changes were proposed in this pull request? This PR reduces # of global variables in generated code by replacing a global variable with a local variable with an allocation of an object every time. When a lot of global variables were generated, the generated code may meet 64K constant pool limit. This PR reduces # of generated global variables in the following three operations: * `Cast` with String to primitive byte/short/int/long * `RegExpReplace` * `CreateArray` I intentionally leave [this part](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala#L595-L603). This is because this variable keeps a class that is dynamically generated. In other word, it is not possible to reuse one class. ## How was this patch tested? Added test cases Author: Kazuaki Ishizaki Closes #19797 from kiszk/SPARK-22570. --- .../spark/sql/catalyst/expressions/Cast.scala | 24 ++++++------- .../expressions/complexTypeCreator.scala | 36 +++++++++++-------- .../expressions/regexpExpressions.scala | 8 ++--- .../sql/catalyst/expressions/CastSuite.scala | 8 +++++ .../expressions/RegexpExpressionsSuite.scala | 11 +++++- .../optimizer/complexTypesSuite.scala | 7 ++++ 6 files changed, 61 insertions(+), 33 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 8cafaef61c7d1..f4ecbdb8393ad 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -799,16 +799,16 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String private[this] def castToByteCode(from: DataType, ctx: CodegenContext): CastFunction = from match { case StringType => - val wrapper = ctx.freshName("wrapper") - ctx.addMutableState("UTF8String.IntWrapper", wrapper, - s"$wrapper = new UTF8String.IntWrapper();") + val wrapper = ctx.freshName("intWrapper") (c, evPrim, evNull) => s""" + UTF8String.IntWrapper $wrapper = new UTF8String.IntWrapper(); if ($c.toByte($wrapper)) { $evPrim = (byte) $wrapper.value; } else { $evNull = true; } + $wrapper = null; """ case BooleanType => (c, evPrim, evNull) => s"$evPrim = $c ? (byte) 1 : (byte) 0;" @@ -826,16 +826,16 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String from: DataType, ctx: CodegenContext): CastFunction = from match { case StringType => - val wrapper = ctx.freshName("wrapper") - ctx.addMutableState("UTF8String.IntWrapper", wrapper, - s"$wrapper = new UTF8String.IntWrapper();") + val wrapper = ctx.freshName("intWrapper") (c, evPrim, evNull) => s""" + UTF8String.IntWrapper $wrapper = new UTF8String.IntWrapper(); if ($c.toShort($wrapper)) { $evPrim = (short) $wrapper.value; } else { $evNull = true; } + $wrapper = null; """ case BooleanType => (c, evPrim, evNull) => s"$evPrim = $c ? (short) 1 : (short) 0;" @@ -851,16 +851,16 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String private[this] def castToIntCode(from: DataType, ctx: CodegenContext): CastFunction = from match { case StringType => - val wrapper = ctx.freshName("wrapper") - ctx.addMutableState("UTF8String.IntWrapper", wrapper, - s"$wrapper = new UTF8String.IntWrapper();") + val wrapper = ctx.freshName("intWrapper") (c, evPrim, evNull) => s""" + UTF8String.IntWrapper $wrapper = new UTF8String.IntWrapper(); if ($c.toInt($wrapper)) { $evPrim = $wrapper.value; } else { $evNull = true; } + $wrapper = null; """ case BooleanType => (c, evPrim, evNull) => s"$evPrim = $c ? 1 : 0;" @@ -876,17 +876,17 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String private[this] def castToLongCode(from: DataType, ctx: CodegenContext): CastFunction = from match { case StringType => - val wrapper = ctx.freshName("wrapper") - ctx.addMutableState("UTF8String.LongWrapper", wrapper, - s"$wrapper = new UTF8String.LongWrapper();") + val wrapper = ctx.freshName("longWrapper") (c, evPrim, evNull) => s""" + UTF8String.LongWrapper $wrapper = new UTF8String.LongWrapper(); if ($c.toLong($wrapper)) { $evPrim = $wrapper.value; } else { $evNull = true; } + $wrapper = null; """ case BooleanType => (c, evPrim, evNull) => s"$evPrim = $c ? 1L : 0L;" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala index 57a7f2e207738..fc68bf478e1c8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala @@ -63,7 +63,7 @@ case class CreateArray(children: Seq[Expression]) extends Expression { val (preprocess, assigns, postprocess, arrayData) = GenArrayData.genCodeToCreateArrayData(ctx, et, evals, false) ev.copy( - code = preprocess + ctx.splitExpressions(assigns) + postprocess, + code = preprocess + assigns + postprocess, value = arrayData, isNull = "false") } @@ -77,24 +77,22 @@ private [sql] object GenArrayData { * * @param ctx a [[CodegenContext]] * @param elementType data type of underlying array elements - * @param elementsCode a set of [[ExprCode]] for each element of an underlying array + * @param elementsCode concatenated set of [[ExprCode]] for each element of an underlying array * @param isMapKey if true, throw an exception when the element is null - * @return (code pre-assignments, assignments to each array elements, code post-assignments, - * arrayData name) + * @return (code pre-assignments, concatenated assignments to each array elements, + * code post-assignments, arrayData name) */ def genCodeToCreateArrayData( ctx: CodegenContext, elementType: DataType, elementsCode: Seq[ExprCode], - isMapKey: Boolean): (String, Seq[String], String, String) = { - val arrayName = ctx.freshName("array") + isMapKey: Boolean): (String, String, String, String) = { val arrayDataName = ctx.freshName("arrayData") val numElements = elementsCode.length if (!ctx.isPrimitiveType(elementType)) { + val arrayName = ctx.freshName("arrayObject") val genericArrayClass = classOf[GenericArrayData].getName - ctx.addMutableState("Object[]", arrayName, - s"$arrayName = new Object[$numElements];") val assignments = elementsCode.zipWithIndex.map { case (eval, i) => val isNullAssignment = if (!isMapKey) { @@ -110,17 +108,21 @@ private [sql] object GenArrayData { } """ } + val assignmentString = ctx.splitExpressions( + expressions = assignments, + funcName = "apply", + extraArguments = ("Object[]", arrayDataName) :: Nil) - ("", - assignments, + (s"Object[] $arrayName = new Object[$numElements];", + assignmentString, s"final ArrayData $arrayDataName = new $genericArrayClass($arrayName);", arrayDataName) } else { + val arrayName = ctx.freshName("array") val unsafeArraySizeInBytes = UnsafeArrayData.calculateHeaderPortionInBytes(numElements) + ByteArrayMethods.roundNumberOfBytesToNearestWord(elementType.defaultSize * numElements) val baseOffset = Platform.BYTE_ARRAY_OFFSET - ctx.addMutableState("UnsafeArrayData", arrayDataName) val primitiveValueTypeName = ctx.primitiveTypeName(elementType) val assignments = elementsCode.zipWithIndex.map { case (eval, i) => @@ -137,14 +139,18 @@ private [sql] object GenArrayData { } """ } + val assignmentString = ctx.splitExpressions( + expressions = assignments, + funcName = "apply", + extraArguments = ("UnsafeArrayData", arrayDataName) :: Nil) (s""" byte[] $arrayName = new byte[$unsafeArraySizeInBytes]; - $arrayDataName = new UnsafeArrayData(); + UnsafeArrayData $arrayDataName = new UnsafeArrayData(); Platform.putLong($arrayName, $baseOffset, $numElements); $arrayDataName.pointTo($arrayName, $baseOffset, $unsafeArraySizeInBytes); """, - assignments, + assignmentString, "", arrayDataName) } @@ -216,10 +222,10 @@ case class CreateMap(children: Seq[Expression]) extends Expression { s""" final boolean ${ev.isNull} = false; $preprocessKeyData - ${ctx.splitExpressions(assignKeys)} + $assignKeys $postprocessKeyData $preprocessValueData - ${ctx.splitExpressions(assignValues)} + $assignValues $postprocessValueData final MapData ${ev.value} = new $mapClass($keyArrayData, $valueArrayData); """ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala index d0d663f63f5db..53d7096dd87d3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala @@ -321,8 +321,7 @@ case class RegExpReplace(subject: Expression, regexp: Expression, rep: Expressio val termLastReplacement = ctx.freshName("lastReplacement") val termLastReplacementInUTF8 = ctx.freshName("lastReplacementInUTF8") - - val termResult = ctx.freshName("result") + val termResult = ctx.freshName("termResult") val classNamePattern = classOf[Pattern].getCanonicalName val classNameStringBuffer = classOf[java.lang.StringBuffer].getCanonicalName @@ -334,8 +333,6 @@ case class RegExpReplace(subject: Expression, regexp: Expression, rep: Expressio ctx.addMutableState("String", termLastReplacement, s"${termLastReplacement} = null;") ctx.addMutableState("UTF8String", termLastReplacementInUTF8, s"${termLastReplacementInUTF8} = null;") - ctx.addMutableState(classNameStringBuffer, - termResult, s"${termResult} = new $classNameStringBuffer();") val setEvNotNull = if (nullable) { s"${ev.isNull} = false;" @@ -355,7 +352,7 @@ case class RegExpReplace(subject: Expression, regexp: Expression, rep: Expressio ${termLastReplacementInUTF8} = $rep.clone(); ${termLastReplacement} = ${termLastReplacementInUTF8}.toString(); } - ${termResult}.delete(0, ${termResult}.length()); + $classNameStringBuffer ${termResult} = new $classNameStringBuffer(); java.util.regex.Matcher ${matcher} = ${termPattern}.matcher($subject.toString()); while (${matcher}.find()) { @@ -363,6 +360,7 @@ case class RegExpReplace(subject: Expression, regexp: Expression, rep: Expressio } ${matcher}.appendTail(${termResult}); ${ev.value} = UTF8String.fromString(${termResult}.toString()); + ${termResult} = null; $setEvNotNull """ }) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala index 7837d6529d12b..65617be05a434 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala @@ -23,6 +23,7 @@ import java.util.{Calendar, Locale, TimeZone} import org.apache.spark.SparkFunSuite import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._ import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.catalyst.util.DateTimeUtils.TimeZoneGMT @@ -845,4 +846,11 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { val outputOuter = Row.fromSeq((1 to N).map(_ => outputInner)) checkEvaluation(cast(Literal.create(inputOuter, fromOuter), toOuter), outputOuter) } + + test("SPARK-22570: Cast should not create a lot of global variables") { + val ctx = new CodegenContext + cast("1", IntegerType).genCode(ctx) + cast("2", LongType).genCode(ctx) + assert(ctx.mutableStates.length == 0) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala index 1ce150e091981..4fa61fbaf66c2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala @@ -20,7 +20,8 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.SparkFunSuite import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.dsl.expressions._ -import org.apache.spark.sql.types.{IntegerType, StringType} +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext +import org.apache.spark.sql.types.StringType /** * Unit tests for regular expression (regexp) related SQL expressions. @@ -178,6 +179,14 @@ class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(nonNullExpr, "num-num", row1) } + test("SPARK-22570: RegExpReplace should not create a lot of global variables") { + val ctx = new CodegenContext + RegExpReplace(Literal("100"), Literal("(\\d+)"), Literal("num")).genCode(ctx) + // four global variables (lastRegex, pattern, lastReplacement, and lastReplacementInUTF8) + // are always required + assert(ctx.mutableStates.length == 4) + } + test("RegexExtract") { val row1 = create_row("100-200", "(\\d+)-(\\d+)", 1) val row2 = create_row("100-200", "(\\d+)-(\\d+)", 2) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala index 3634accf1ec21..e3675367d78e4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/complexTypesSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.optimizer import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, Range} import org.apache.spark.sql.catalyst.rules.RuleExecutor @@ -164,6 +165,12 @@ class ComplexTypesSuite extends PlanTest{ comparePlans(Optimizer execute query, expected) } + test("SPARK-22570: CreateArray should not create a lot of global variables") { + val ctx = new CodegenContext + CreateArray(Seq(Literal(1))).genCode(ctx) + assert(ctx.mutableStates.length == 0) + } + test("simplify map ops") { val rel = relation .select( From 6ac57fd0d1c82b834eb4bf0dd57596b92a99d6de Mon Sep 17 00:00:00 2001 From: aokolnychyi Date: Thu, 30 Nov 2017 14:25:10 -0800 Subject: [PATCH 02/37] [SPARK-21417][SQL] Infer join conditions using propagated constraints ## What changes were proposed in this pull request? This PR adds an optimization rule that infers join conditions using propagated constraints. For instance, if there is a join, where the left relation has 'a = 1' and the right relation has 'b = 1', then the rule infers 'a = b' as a join predicate. Only semantically new predicates are appended to the existing join condition. Refer to the corresponding ticket and tests for more details. ## How was this patch tested? This patch comes with a new test suite to cover the implemented logic. Author: aokolnychyi Closes #18692 from aokolnychyi/spark-21417. --- .../expressions/EquivalentExpressionMap.scala | 66 +++++ .../catalyst/expressions/ExpressionSet.scala | 2 + .../sql/catalyst/optimizer/Optimizer.scala | 1 + .../spark/sql/catalyst/optimizer/joins.scala | 60 +++++ .../EquivalentExpressionMapSuite.scala | 56 +++++ .../optimizer/EliminateCrossJoinSuite.scala | 238 ++++++++++++++++++ 6 files changed, 423 insertions(+) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressionMap.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressionMapSuite.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateCrossJoinSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressionMap.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressionMap.scala new file mode 100644 index 0000000000000..cf1614afb1a76 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressionMap.scala @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions + +import scala.collection.mutable + +import org.apache.spark.sql.catalyst.expressions.EquivalentExpressionMap.SemanticallyEqualExpr + +/** + * A class that allows you to map an expression into a set of equivalent expressions. The keys are + * handled based on their semantic meaning and ignoring cosmetic differences. The values are + * represented as [[ExpressionSet]]s. + * + * The underlying representation of keys depends on the [[Expression.semanticHash]] and + * [[Expression.semanticEquals]] methods. + * + * {{{ + * val map = new EquivalentExpressionMap() + * + * map.put(1 + 2, a) + * map.put(rand(), b) + * + * map.get(2 + 1) => Set(a) // 1 + 2 and 2 + 1 are semantically equivalent + * map.get(1 + 2) => Set(a) // 1 + 2 and 2 + 1 are semantically equivalent + * map.get(rand()) => Set() // non-deterministic expressions are not equivalent + * }}} + */ +class EquivalentExpressionMap { + + private val equivalenceMap = mutable.HashMap.empty[SemanticallyEqualExpr, ExpressionSet] + + def put(expression: Expression, equivalentExpression: Expression): Unit = { + val equivalentExpressions = equivalenceMap.getOrElseUpdate(expression, ExpressionSet.empty) + equivalenceMap(expression) = equivalentExpressions + equivalentExpression + } + + def get(expression: Expression): Set[Expression] = + equivalenceMap.getOrElse(expression, ExpressionSet.empty) +} + +object EquivalentExpressionMap { + + private implicit class SemanticallyEqualExpr(val expr: Expression) { + override def equals(obj: Any): Boolean = obj match { + case other: SemanticallyEqualExpr => expr.semanticEquals(other.expr) + case _ => false + } + + override def hashCode: Int = expr.semanticHash() + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala index 7e8e7b8cd5f18..e9890837af07d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala @@ -27,6 +27,8 @@ object ExpressionSet { expressions.foreach(set.add) set } + + val empty: ExpressionSet = ExpressionSet(Nil) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 0d961bf2e6e5e..8a5c486912abf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -87,6 +87,7 @@ abstract class Optimizer(sessionCatalog: SessionCatalog) PushProjectionThroughUnion, ReorderJoin, EliminateOuterJoin, + EliminateCrossJoin, InferFiltersFromConstraints, BooleanSimplification, PushPredicateThroughJoin, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala index edbeaf273fd6f..29a3a7f109b80 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.optimizer import scala.annotation.tailrec +import scala.collection.mutable import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.ExtractFiltersAndInnerJoins @@ -152,3 +153,62 @@ object EliminateOuterJoin extends Rule[LogicalPlan] with PredicateHelper { if (j.joinType == newJoinType) f else Filter(condition, j.copy(joinType = newJoinType)) } } + +/** + * A rule that eliminates CROSS joins by inferring join conditions from propagated constraints. + * + * The optimization is applicable only to CROSS joins. For other join types, adding inferred join + * conditions would potentially shuffle children as child node's partitioning won't satisfy the JOIN + * node's requirements which otherwise could have. + * + * For instance, given a CROSS join with the constraint 'a = 1' from the left child and the + * constraint 'b = 1' from the right child, this rule infers a new join predicate 'a = b' and + * converts it to an Inner join. + */ +object EliminateCrossJoin extends Rule[LogicalPlan] with PredicateHelper { + + def apply(plan: LogicalPlan): LogicalPlan = { + if (SQLConf.get.constraintPropagationEnabled) { + eliminateCrossJoin(plan) + } else { + plan + } + } + + private def eliminateCrossJoin(plan: LogicalPlan): LogicalPlan = plan transform { + case join @ Join(leftPlan, rightPlan, Cross, None) => + val leftConstraints = join.constraints.filter(_.references.subsetOf(leftPlan.outputSet)) + val rightConstraints = join.constraints.filter(_.references.subsetOf(rightPlan.outputSet)) + val inferredJoinPredicates = inferJoinPredicates(leftConstraints, rightConstraints) + val joinConditionOpt = inferredJoinPredicates.reduceOption(And) + if (joinConditionOpt.isDefined) Join(leftPlan, rightPlan, Inner, joinConditionOpt) else join + } + + private def inferJoinPredicates( + leftConstraints: Set[Expression], + rightConstraints: Set[Expression]): mutable.Set[EqualTo] = { + + val equivalentExpressionMap = new EquivalentExpressionMap() + + leftConstraints.foreach { + case EqualTo(attr: Attribute, expr: Expression) => + equivalentExpressionMap.put(expr, attr) + case EqualTo(expr: Expression, attr: Attribute) => + equivalentExpressionMap.put(expr, attr) + case _ => + } + + val joinConditions = mutable.Set.empty[EqualTo] + + rightConstraints.foreach { + case EqualTo(attr: Attribute, expr: Expression) => + joinConditions ++= equivalentExpressionMap.get(expr).map(EqualTo(attr, _)) + case EqualTo(expr: Expression, attr: Attribute) => + joinConditions ++= equivalentExpressionMap.get(expr).map(EqualTo(attr, _)) + case _ => + } + + joinConditions + } + +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressionMapSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressionMapSuite.scala new file mode 100644 index 0000000000000..bad7e17bb6cf2 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressionMapSuite.scala @@ -0,0 +1,56 @@ +/* + * 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.sql.catalyst.expressions + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.dsl.expressions._ + +class EquivalentExpressionMapSuite extends SparkFunSuite { + + private val onePlusTwo = Literal(1) + Literal(2) + private val twoPlusOne = Literal(2) + Literal(1) + private val rand = Rand(10) + + test("behaviour of the equivalent expression map") { + val equivalentExpressionMap = new EquivalentExpressionMap() + equivalentExpressionMap.put(onePlusTwo, 'a) + equivalentExpressionMap.put(Literal(1) + Literal(3), 'b) + equivalentExpressionMap.put(rand, 'c) + + // 1 + 2 should be equivalent to 2 + 1 + assertResult(ExpressionSet(Seq('a)))(equivalentExpressionMap.get(twoPlusOne)) + // non-deterministic expressions should not be equivalent + assertResult(ExpressionSet.empty)(equivalentExpressionMap.get(rand)) + + // if the same (key, value) is added several times, the map still returns only one entry + equivalentExpressionMap.put(onePlusTwo, 'a) + equivalentExpressionMap.put(twoPlusOne, 'a) + assertResult(ExpressionSet(Seq('a)))(equivalentExpressionMap.get(twoPlusOne)) + + // get several equivalent attributes + equivalentExpressionMap.put(onePlusTwo, 'e) + assertResult(ExpressionSet(Seq('a, 'e)))(equivalentExpressionMap.get(onePlusTwo)) + assertResult(2)(equivalentExpressionMap.get(onePlusTwo).size) + + // several non-deterministic expressions should not be equivalent + equivalentExpressionMap.put(rand, 'd) + assertResult(ExpressionSet.empty)(equivalentExpressionMap.get(rand)) + assertResult(0)(equivalentExpressionMap.get(rand).size) + } + +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateCrossJoinSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateCrossJoinSuite.scala new file mode 100644 index 0000000000000..e04dd28ee36a0 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateCrossJoinSuite.scala @@ -0,0 +1,238 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.optimizer + +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.expressions.{Cast, Expression, Literal, Not, Rand} +import org.apache.spark.sql.catalyst.plans.{Cross, Inner, JoinType, PlanTest} +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} +import org.apache.spark.sql.catalyst.rules.RuleExecutor +import org.apache.spark.sql.internal.SQLConf.CONSTRAINT_PROPAGATION_ENABLED +import org.apache.spark.sql.types.IntegerType + +class EliminateCrossJoinSuite extends PlanTest { + + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = + Batch("Eliminate cross joins", FixedPoint(10), + EliminateCrossJoin, + PushPredicateThroughJoin) :: Nil + } + + val testRelation1 = LocalRelation('a.int, 'b.int) + val testRelation2 = LocalRelation('c.int, 'd.int) + + test("successful elimination of cross joins (1)") { + checkJoinOptimization( + originalFilter = 'a === 1 && 'c === 1 && 'd === 1, + originalJoinType = Cross, + originalJoinCondition = None, + expectedFilter = None, + expectedLeftRelationFilter = 'a === 1, + expectedRightRelationFilter = 'c === 1 && 'd === 1, + expectedJoinType = Inner, + expectedJoinCondition = Some('a === 'c && 'a === 'd)) + } + + test("successful elimination of cross joins (2)") { + checkJoinOptimization( + originalFilter = 'a === 1 && 'b === 2 && 'd === 1, + originalJoinType = Cross, + originalJoinCondition = None, + expectedFilter = None, + expectedLeftRelationFilter = 'a === 1 && 'b === 2, + expectedRightRelationFilter = 'd === 1, + expectedJoinType = Inner, + expectedJoinCondition = Some('a === 'd)) + } + + test("successful elimination of cross joins (3)") { + // PushPredicateThroughJoin will push 'd === 'a into the join condition + // EliminateCrossJoin will NOT apply because the condition will be already present + // therefore, the join type will stay the same (i.e., CROSS) + checkJoinOptimization( + originalFilter = 'a === 1 && Literal(1) === 'd && 'd === 'a, + originalJoinType = Cross, + originalJoinCondition = None, + expectedFilter = None, + expectedLeftRelationFilter = 'a === 1, + expectedRightRelationFilter = Literal(1) === 'd, + expectedJoinType = Cross, + expectedJoinCondition = Some('a === 'd)) + } + + test("successful elimination of cross joins (4)") { + // Literal(1) * Literal(2) and Literal(2) * Literal(1) are semantically equal + checkJoinOptimization( + originalFilter = 'a === Literal(1) * Literal(2) && Literal(2) * Literal(1) === 'c, + originalJoinType = Cross, + originalJoinCondition = None, + expectedFilter = None, + expectedLeftRelationFilter = 'a === Literal(1) * Literal(2), + expectedRightRelationFilter = Literal(2) * Literal(1) === 'c, + expectedJoinType = Inner, + expectedJoinCondition = Some('a === 'c)) + } + + test("successful elimination of cross joins (5)") { + checkJoinOptimization( + originalFilter = 'a === 1 && Literal(1) === 'a && 'c === 1, + originalJoinType = Cross, + originalJoinCondition = None, + expectedFilter = None, + expectedLeftRelationFilter = 'a === 1 && Literal(1) === 'a, + expectedRightRelationFilter = 'c === 1, + expectedJoinType = Inner, + expectedJoinCondition = Some('a === 'c)) + } + + test("successful elimination of cross joins (6)") { + checkJoinOptimization( + originalFilter = 'a === Cast("1", IntegerType) && 'c === Cast("1", IntegerType) && 'd === 1, + originalJoinType = Cross, + originalJoinCondition = None, + expectedFilter = None, + expectedLeftRelationFilter = 'a === Cast("1", IntegerType), + expectedRightRelationFilter = 'c === Cast("1", IntegerType) && 'd === 1, + expectedJoinType = Inner, + expectedJoinCondition = Some('a === 'c)) + } + + test("successful elimination of cross joins (7)") { + // The join condition appears due to PushPredicateThroughJoin + checkJoinOptimization( + originalFilter = (('a >= 1 && 'c === 1) || 'd === 10) && 'b === 10 && 'c === 1, + originalJoinType = Cross, + originalJoinCondition = None, + expectedFilter = None, + expectedLeftRelationFilter = 'b === 10, + expectedRightRelationFilter = 'c === 1, + expectedJoinType = Cross, + expectedJoinCondition = Some(('a >= 1 && 'c === 1) || 'd === 10)) + } + + test("successful elimination of cross joins (8)") { + checkJoinOptimization( + originalFilter = 'a === 1 && 'c === 1 && Literal(1) === 'a && Literal(1) === 'c, + originalJoinType = Cross, + originalJoinCondition = None, + expectedFilter = None, + expectedLeftRelationFilter = 'a === 1 && Literal(1) === 'a, + expectedRightRelationFilter = 'c === 1 && Literal(1) === 'c, + expectedJoinType = Inner, + expectedJoinCondition = Some('a === 'c)) + } + + test("inability to detect join conditions when constant propagation is disabled") { + withSQLConf(CONSTRAINT_PROPAGATION_ENABLED.key -> "false") { + checkJoinOptimization( + originalFilter = 'a === 1 && 'c === 1 && 'd === 1, + originalJoinType = Cross, + originalJoinCondition = None, + expectedFilter = None, + expectedLeftRelationFilter = 'a === 1, + expectedRightRelationFilter = 'c === 1 && 'd === 1, + expectedJoinType = Cross, + expectedJoinCondition = None) + } + } + + test("inability to detect join conditions (1)") { + checkJoinOptimization( + originalFilter = 'a >= 1 && 'c === 1 && 'd >= 1, + originalJoinType = Cross, + originalJoinCondition = None, + expectedFilter = None, + expectedLeftRelationFilter = 'a >= 1, + expectedRightRelationFilter = 'c === 1 && 'd >= 1, + expectedJoinType = Cross, + expectedJoinCondition = None) + } + + test("inability to detect join conditions (2)") { + checkJoinOptimization( + originalFilter = Literal(1) === 'b && ('c === 1 || 'd === 1), + originalJoinType = Cross, + originalJoinCondition = None, + expectedFilter = None, + expectedLeftRelationFilter = Literal(1) === 'b, + expectedRightRelationFilter = 'c === 1 || 'd === 1, + expectedJoinType = Cross, + expectedJoinCondition = None) + } + + test("inability to detect join conditions (3)") { + checkJoinOptimization( + originalFilter = Literal(1) === 'b && 'c === 1, + originalJoinType = Cross, + originalJoinCondition = Some('c === 'b), + expectedFilter = None, + expectedLeftRelationFilter = Literal(1) === 'b, + expectedRightRelationFilter = 'c === 1, + expectedJoinType = Cross, + expectedJoinCondition = Some('c === 'b)) + } + + test("inability to detect join conditions (4)") { + checkJoinOptimization( + originalFilter = Not('a === 1) && 'd === 1, + originalJoinType = Cross, + originalJoinCondition = None, + expectedFilter = None, + expectedLeftRelationFilter = Not('a === 1), + expectedRightRelationFilter = 'd === 1, + expectedJoinType = Cross, + expectedJoinCondition = None) + } + + test("inability to detect join conditions (5)") { + checkJoinOptimization( + originalFilter = 'a === Rand(10) && 'b === 1 && 'd === Rand(10) && 'c === 3, + originalJoinType = Cross, + originalJoinCondition = None, + expectedFilter = Some('a === Rand(10) && 'd === Rand(10)), + expectedLeftRelationFilter = 'b === 1, + expectedRightRelationFilter = 'c === 3, + expectedJoinType = Cross, + expectedJoinCondition = None) + } + + private def checkJoinOptimization( + originalFilter: Expression, + originalJoinType: JoinType, + originalJoinCondition: Option[Expression], + expectedFilter: Option[Expression], + expectedLeftRelationFilter: Expression, + expectedRightRelationFilter: Expression, + expectedJoinType: JoinType, + expectedJoinCondition: Option[Expression]): Unit = { + + val originalQuery = testRelation1 + .join(testRelation2, originalJoinType, originalJoinCondition) + .where(originalFilter) + val optimizedQuery = Optimize.execute(originalQuery.analyze) + + val left = testRelation1.where(expectedLeftRelationFilter) + val right = testRelation2.where(expectedRightRelationFilter) + val join = left.join(right, expectedJoinType, expectedJoinCondition) + val expectedQuery = expectedFilter.fold(join)(join.where(_)).analyze + + comparePlans(optimizedQuery, expectedQuery) + } +} From bcceab649510a45f4c4b8e44b157c9987adff6f4 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Thu, 30 Nov 2017 15:36:26 -0800 Subject: [PATCH 03/37] [SPARK-22489][SQL] Shouldn't change broadcast join buildSide if user clearly specified ## What changes were proposed in this pull request? How to reproduce: ```scala import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec spark.createDataFrame(Seq((1, "4"), (2, "2"))).toDF("key", "value").createTempView("table1") spark.createDataFrame(Seq((1, "1"), (2, "2"))).toDF("key", "value").createTempView("table2") val bl = sql("SELECT /*+ MAPJOIN(t1) */ * FROM table1 t1 JOIN table2 t2 ON t1.key = t2.key").queryExecution.executedPlan println(bl.children.head.asInstanceOf[BroadcastHashJoinExec].buildSide) ``` The result is `BuildRight`, but should be `BuildLeft`. This PR fix this issue. ## How was this patch tested? unit tests Author: Yuming Wang Closes #19714 from wangyum/SPARK-22489. --- docs/sql-programming-guide.md | 58 ++++++++++++++++ .../spark/sql/execution/SparkStrategies.scala | 67 +++++++++++++----- .../execution/joins/BroadcastJoinSuite.scala | 69 ++++++++++++++++++- 3 files changed, 177 insertions(+), 17 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 983770d506836..a1b9c3bbfd059 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1492,6 +1492,64 @@ that these options will be deprecated in future release as more optimizations ar +## Broadcast Hint for SQL Queries + +The `BROADCAST` hint guides Spark to broadcast each specified table when joining them with another table or view. +When Spark deciding the join methods, the broadcast hash join (i.e., BHJ) is preferred, +even if the statistics is above the configuration `spark.sql.autoBroadcastJoinThreshold`. +When both sides of a join are specified, Spark broadcasts the one having the lower statistics. +Note Spark does not guarantee BHJ is always chosen, since not all cases (e.g. full outer join) +support BHJ. When the broadcast nested loop join is selected, we still respect the hint. + +
+ +
+ +{% highlight scala %} +import org.apache.spark.sql.functions.broadcast +broadcast(spark.table("src")).join(spark.table("records"), "key").show() +{% endhighlight %} + +
+ +
+ +{% highlight java %} +import static org.apache.spark.sql.functions.broadcast; +broadcast(spark.table("src")).join(spark.table("records"), "key").show(); +{% endhighlight %} + +
+ +
+ +{% highlight python %} +from pyspark.sql.functions import broadcast +broadcast(spark.table("src")).join(spark.table("records"), "key").show() +{% endhighlight %} + +
+ +
+ +{% highlight r %} +src <- sql("SELECT * FROM src") +records <- sql("SELECT * FROM records") +head(join(broadcast(src), records, src$key == records$key)) +{% endhighlight %} + +
+ +
+ +{% highlight sql %} +-- We accept BROADCAST, BROADCASTJOIN and MAPJOIN for broadcast hint +SELECT /*+ BROADCAST(r) */ * FROM records r JOIN src s ON r.key = s.key +{% endhighlight %} + +
+
+ # Distributed SQL Engine Spark SQL can also act as a distributed query engine using its JDBC/ODBC or command-line interface. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 19b858faba6ea..1fe3cb1c8750a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution.columnar.{InMemoryRelation, InMemoryTableScanExec} import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec -import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight} +import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.StreamingQuery @@ -91,12 +91,12 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { * predicates can be evaluated by matching join keys. If found, Join implementations are chosen * with the following precedence: * - * - Broadcast: if one side of the join has an estimated physical size that is smaller than the - * user-configurable [[SQLConf.AUTO_BROADCASTJOIN_THRESHOLD]] threshold - * or if that side has an explicit broadcast hint (e.g. the user applied the - * [[org.apache.spark.sql.functions.broadcast()]] function to a DataFrame), then that side - * of the join will be broadcasted and the other side will be streamed, with no shuffling - * performed. If both sides of the join are eligible to be broadcasted then the + * - Broadcast: We prefer to broadcast the join side with an explicit broadcast hint(e.g. the + * user applied the [[org.apache.spark.sql.functions.broadcast()]] function to a DataFrame). + * If both sides have the broadcast hint, we prefer to broadcast the side with a smaller + * estimated physical size. If neither one of the sides has the broadcast hint, + * we only broadcast the join side if its estimated physical size that is smaller than + * the user-configurable [[SQLConf.AUTO_BROADCASTJOIN_THRESHOLD]] threshold. * - Shuffle hash join: if the average size of a single partition is small enough to build a hash * table. * - Sort merge: if the matching join keys are sortable. @@ -112,9 +112,7 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { * Matches a plan whose output should be small enough to be used in broadcast join. */ private def canBroadcast(plan: LogicalPlan): Boolean = { - plan.stats.hints.broadcast || - (plan.stats.sizeInBytes >= 0 && - plan.stats.sizeInBytes <= conf.autoBroadcastJoinThreshold) + plan.stats.sizeInBytes >= 0 && plan.stats.sizeInBytes <= conf.autoBroadcastJoinThreshold } /** @@ -149,10 +147,45 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case _ => false } + private def broadcastSide( + canBuildLeft: Boolean, + canBuildRight: Boolean, + left: LogicalPlan, + right: LogicalPlan): BuildSide = { + + def smallerSide = + if (right.stats.sizeInBytes <= left.stats.sizeInBytes) BuildRight else BuildLeft + + val buildRight = canBuildRight && right.stats.hints.broadcast + val buildLeft = canBuildLeft && left.stats.hints.broadcast + + if (buildRight && buildLeft) { + // Broadcast smaller side base on its estimated physical size + // if both sides have broadcast hint + smallerSide + } else if (buildRight) { + BuildRight + } else if (buildLeft) { + BuildLeft + } else if (canBuildRight && canBuildLeft) { + // for the last default broadcast nested loop join + smallerSide + } else { + throw new AnalysisException("Can not decide which side to broadcast for this join") + } + } + def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { // --- BroadcastHashJoin -------------------------------------------------------------------- + case ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, condition, left, right) + if (canBuildRight(joinType) && right.stats.hints.broadcast) || + (canBuildLeft(joinType) && left.stats.hints.broadcast) => + val buildSide = broadcastSide(canBuildLeft(joinType), canBuildRight(joinType), left, right) + Seq(joins.BroadcastHashJoinExec( + leftKeys, rightKeys, joinType, buildSide, condition, planLater(left), planLater(right))) + case ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, condition, left, right) if canBuildRight(joinType) && canBroadcast(right) => Seq(joins.BroadcastHashJoinExec( @@ -189,6 +222,13 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { // --- Without joining keys ------------------------------------------------------------ // Pick BroadcastNestedLoopJoin if one side could be broadcasted + case j @ logical.Join(left, right, joinType, condition) + if (canBuildRight(joinType) && right.stats.hints.broadcast) || + (canBuildLeft(joinType) && left.stats.hints.broadcast) => + val buildSide = broadcastSide(canBuildLeft(joinType), canBuildRight(joinType), left, right) + joins.BroadcastNestedLoopJoinExec( + planLater(left), planLater(right), buildSide, joinType, condition) :: Nil + case j @ logical.Join(left, right, joinType, condition) if canBuildRight(joinType) && canBroadcast(right) => joins.BroadcastNestedLoopJoinExec( @@ -203,12 +243,7 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { joins.CartesianProductExec(planLater(left), planLater(right), condition) :: Nil case logical.Join(left, right, joinType, condition) => - val buildSide = - if (right.stats.sizeInBytes <= left.stats.sizeInBytes) { - BuildRight - } else { - BuildLeft - } + val buildSide = broadcastSide(canBuildLeft = true, canBuildRight = true, left, right) // This join could be very slow or OOM joins.BroadcastNestedLoopJoinExec( planLater(left), planLater(right), buildSide, joinType, condition) :: Nil diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala index a0fad862b44c7..67e2cdc7394bb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala @@ -22,7 +22,7 @@ import scala.reflect.ClassTag import org.apache.spark.AccumulatorSuite import org.apache.spark.sql.{Dataset, QueryTest, Row, SparkSession} import org.apache.spark.sql.catalyst.expressions.{BitwiseAnd, BitwiseOr, Cast, Literal, ShiftLeft} -import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.{BinaryExecNode, SparkPlan, WholeStageCodegenExec} import org.apache.spark.sql.execution.exchange.EnsureRequirements import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf @@ -223,4 +223,71 @@ class BroadcastJoinSuite extends QueryTest with SQLTestUtils { assert(HashJoin.rewriteKeyExpr(l :: ss :: Nil) === l :: ss :: Nil) assert(HashJoin.rewriteKeyExpr(i :: ss :: Nil) === i :: ss :: Nil) } + + test("Shouldn't change broadcast join buildSide if user clearly specified") { + def assertJoinBuildSide(sqlStr: String, joinMethod: String, buildSide: BuildSide): Any = { + val executedPlan = sql(sqlStr).queryExecution.executedPlan + executedPlan match { + case b: BroadcastNestedLoopJoinExec => + assert(b.getClass.getSimpleName === joinMethod) + assert(b.buildSide === buildSide) + case w: WholeStageCodegenExec => + assert(w.children.head.getClass.getSimpleName === joinMethod) + assert(w.children.head.asInstanceOf[BroadcastHashJoinExec].buildSide === buildSide) + } + } + + withTempView("t1", "t2") { + spark.createDataFrame(Seq((1, "4"), (2, "2"))).toDF("key", "value").createTempView("t1") + spark.createDataFrame(Seq((1, "1"), (2, "12.3"), (2, "123"))).toDF("key", "value") + .createTempView("t2") + + val t1Size = spark.table("t1").queryExecution.analyzed.children.head.stats.sizeInBytes + val t2Size = spark.table("t2").queryExecution.analyzed.children.head.stats.sizeInBytes + assert(t1Size < t2Size) + + val bh = BroadcastHashJoinExec.toString + val bl = BroadcastNestedLoopJoinExec.toString + + // INNER JOIN && t1Size < t2Size => BuildLeft + assertJoinBuildSide( + "SELECT /*+ MAPJOIN(t1, t2) */ * FROM t1 JOIN t2 ON t1.key = t2.key", bh, BuildLeft) + // LEFT JOIN => BuildRight + assertJoinBuildSide( + "SELECT /*+ MAPJOIN(t1, t2) */ * FROM t1 LEFT JOIN t2 ON t1.key = t2.key", bh, BuildRight) + // RIGHT JOIN => BuildLeft + assertJoinBuildSide( + "SELECT /*+ MAPJOIN(t1, t2) */ * FROM t1 RIGHT JOIN t2 ON t1.key = t2.key", bh, BuildLeft) + // INNER JOIN && broadcast(t1) => BuildLeft + assertJoinBuildSide( + "SELECT /*+ MAPJOIN(t1) */ * FROM t1 JOIN t2 ON t1.key = t2.key", bh, BuildLeft) + // INNER JOIN && broadcast(t2) => BuildRight + assertJoinBuildSide( + "SELECT /*+ MAPJOIN(t2) */ * FROM t1 JOIN t2 ON t1.key = t2.key", bh, BuildRight) + + + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "0", + SQLConf.CROSS_JOINS_ENABLED.key -> "true") { + // INNER JOIN && t1Size < t2Size => BuildLeft + assertJoinBuildSide("SELECT /*+ MAPJOIN(t1, t2) */ * FROM t1 JOIN t2", bl, BuildLeft) + // FULL JOIN && t1Size < t2Size => BuildLeft + assertJoinBuildSide("SELECT /*+ MAPJOIN(t1, t2) */ * FROM t1 FULL JOIN t2", bl, BuildLeft) + // LEFT JOIN => BuildRight + assertJoinBuildSide("SELECT /*+ MAPJOIN(t1, t2) */ * FROM t1 LEFT JOIN t2", bl, BuildRight) + // RIGHT JOIN => BuildLeft + assertJoinBuildSide("SELECT /*+ MAPJOIN(t1, t2) */ * FROM t1 RIGHT JOIN t2", bl, BuildLeft) + // INNER JOIN && broadcast(t1) => BuildLeft + assertJoinBuildSide("SELECT /*+ MAPJOIN(t1) */ * FROM t1 JOIN t2", bl, BuildLeft) + // INNER JOIN && broadcast(t2) => BuildRight + assertJoinBuildSide("SELECT /*+ MAPJOIN(t2) */ * FROM t1 JOIN t2", bl, BuildRight) + // FULL OUTER && broadcast(t1) => BuildLeft + assertJoinBuildSide("SELECT /*+ MAPJOIN(t1) */ * FROM t1 FULL OUTER JOIN t2", bl, BuildLeft) + // FULL OUTER && broadcast(t2) => BuildRight + assertJoinBuildSide( + "SELECT /*+ MAPJOIN(t2) */ * FROM t1 FULL OUTER JOIN t2", bl, BuildRight) + // FULL OUTER && t1Size < t2Size => BuildLeft + assertJoinBuildSide("SELECT * FROM t1 FULL OUTER JOIN t2", bl, BuildLeft) + } + } + } } From f5f8e84d9d35751dad51490b6ae22931aa88db7b Mon Sep 17 00:00:00 2001 From: Adrian Ionescu Date: Thu, 30 Nov 2017 15:41:34 -0800 Subject: [PATCH 04/37] [SPARK-22614] Dataset API: repartitionByRange(...) ## What changes were proposed in this pull request? This PR introduces a way to explicitly range-partition a Dataset. So far, only round-robin and hash partitioning were possible via `df.repartition(...)`, but sometimes range partitioning might be desirable: e.g. when writing to disk, for better compression without the cost of global sort. The current implementation piggybacks on the existing `RepartitionByExpression` `LogicalPlan` and simply adds the following logic: If its expressions are of type `SortOrder`, then it will do `RangePartitioning`; otherwise `HashPartitioning`. This was by far the least intrusive solution I could come up with. ## How was this patch tested? Unit test for `RepartitionByExpression` changes, a test to ensure we're not changing the behavior of existing `.repartition()` and a few end-to-end tests in `DataFrameSuite`. Author: Adrian Ionescu Closes #19828 from adrian-ionescu/repartitionByRange. --- .../plans/logical/basicLogicalOperators.scala | 20 +++++++ .../sql/catalyst/analysis/AnalysisSuite.scala | 26 +++++++++ .../scala/org/apache/spark/sql/Dataset.scala | 57 +++++++++++++++++-- .../spark/sql/execution/SparkStrategies.scala | 5 +- .../org/apache/spark/sql/DataFrameSuite.scala | 57 +++++++++++++++++++ 5 files changed, 157 insertions(+), 8 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index c2750c3079814..93de7c1daf5c2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.statsEstimation._ +import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, RangePartitioning} import org.apache.spark.sql.types._ import org.apache.spark.util.Utils import org.apache.spark.util.random.RandomSampler @@ -838,6 +839,25 @@ case class RepartitionByExpression( require(numPartitions > 0, s"Number of partitions ($numPartitions) must be positive.") + val partitioning: Partitioning = { + val (sortOrder, nonSortOrder) = partitionExpressions.partition(_.isInstanceOf[SortOrder]) + + require(sortOrder.isEmpty || nonSortOrder.isEmpty, + s"${getClass.getSimpleName} expects that either all its `partitionExpressions` are of type " + + "`SortOrder`, which means `RangePartitioning`, or none of them are `SortOrder`, which " + + "means `HashPartitioning`. In this case we have:" + + s""" + |SortOrder: ${sortOrder} + |NonSortOrder: ${nonSortOrder} + """.stripMargin) + + if (sortOrder.nonEmpty) { + RangePartitioning(sortOrder.map(_.asInstanceOf[SortOrder]), numPartitions) + } else { + HashPartitioning(nonSortOrder, numPartitions) + } + } + override def maxRows: Option[Long] = child.maxRows override def shuffle: Boolean = true } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index e56a5d6368318..0e2e706a31a05 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.{Cross, Inner} import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, RangePartitioning} import org.apache.spark.sql.types._ @@ -514,4 +515,29 @@ class AnalysisSuite extends AnalysisTest with Matchers { Seq("Number of column aliases does not match number of columns. " + "Number of column aliases: 5; number of columns: 4.")) } + + test("SPARK-22614 RepartitionByExpression partitioning") { + def checkPartitioning[T <: Partitioning](numPartitions: Int, exprs: Expression*): Unit = { + val partitioning = RepartitionByExpression(exprs, testRelation2, numPartitions).partitioning + assert(partitioning.isInstanceOf[T]) + } + + checkPartitioning[HashPartitioning](numPartitions = 10, exprs = Literal(20)) + checkPartitioning[HashPartitioning](numPartitions = 10, exprs = 'a.attr, 'b.attr) + + checkPartitioning[RangePartitioning](numPartitions = 10, + exprs = SortOrder(Literal(10), Ascending)) + checkPartitioning[RangePartitioning](numPartitions = 10, + exprs = SortOrder('a.attr, Ascending), SortOrder('b.attr, Descending)) + + intercept[IllegalArgumentException] { + checkPartitioning(numPartitions = 0, exprs = Literal(20)) + } + intercept[IllegalArgumentException] { + checkPartitioning(numPartitions = -1, exprs = Literal(20)) + } + intercept[IllegalArgumentException] { + checkPartitioning(numPartitions = 10, exprs = SortOrder('a.attr, Ascending), 'b.attr) + } + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 1620ab3aa2094..167c9d050c3c4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -2732,8 +2732,18 @@ class Dataset[T] private[sql]( * @since 2.0.0 */ @scala.annotation.varargs - def repartition(numPartitions: Int, partitionExprs: Column*): Dataset[T] = withTypedPlan { - RepartitionByExpression(partitionExprs.map(_.expr), logicalPlan, numPartitions) + def repartition(numPartitions: Int, partitionExprs: Column*): Dataset[T] = { + // The underlying `LogicalPlan` operator special-cases all-`SortOrder` arguments. + // However, we don't want to complicate the semantics of this API method. + // Instead, let's give users a friendly error message, pointing them to the new method. + val sortOrders = partitionExprs.filter(_.expr.isInstanceOf[SortOrder]) + if (sortOrders.nonEmpty) throw new IllegalArgumentException( + s"""Invalid partitionExprs specified: $sortOrders + |For range partitioning use repartitionByRange(...) instead. + """.stripMargin) + withTypedPlan { + RepartitionByExpression(partitionExprs.map(_.expr), logicalPlan, numPartitions) + } } /** @@ -2747,9 +2757,46 @@ class Dataset[T] private[sql]( * @since 2.0.0 */ @scala.annotation.varargs - def repartition(partitionExprs: Column*): Dataset[T] = withTypedPlan { - RepartitionByExpression( - partitionExprs.map(_.expr), logicalPlan, sparkSession.sessionState.conf.numShufflePartitions) + def repartition(partitionExprs: Column*): Dataset[T] = { + repartition(sparkSession.sessionState.conf.numShufflePartitions, partitionExprs: _*) + } + + /** + * Returns a new Dataset partitioned by the given partitioning expressions into + * `numPartitions`. The resulting Dataset is range partitioned. + * + * At least one partition-by expression must be specified. + * When no explicit sort order is specified, "ascending nulls first" is assumed. + * + * @group typedrel + * @since 2.3.0 + */ + @scala.annotation.varargs + def repartitionByRange(numPartitions: Int, partitionExprs: Column*): Dataset[T] = { + require(partitionExprs.nonEmpty, "At least one partition-by expression must be specified.") + val sortOrder: Seq[SortOrder] = partitionExprs.map(_.expr match { + case expr: SortOrder => expr + case expr: Expression => SortOrder(expr, Ascending) + }) + withTypedPlan { + RepartitionByExpression(sortOrder, logicalPlan, numPartitions) + } + } + + /** + * Returns a new Dataset partitioned by the given partitioning expressions, using + * `spark.sql.shuffle.partitions` as number of partitions. + * The resulting Dataset is range partitioned. + * + * At least one partition-by expression must be specified. + * When no explicit sort order is specified, "ascending nulls first" is assumed. + * + * @group typedrel + * @since 2.3.0 + */ + @scala.annotation.varargs + def repartitionByRange(partitionExprs: Column*): Dataset[T] = { + repartitionByRange(sparkSession.sessionState.conf.numShufflePartitions, partitionExprs: _*) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 1fe3cb1c8750a..9e713cd7bbe2b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -482,9 +482,8 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { execution.RDDScanExec(Nil, singleRowRdd, "OneRowRelation") :: Nil case r: logical.Range => execution.RangeExec(r) :: Nil - case logical.RepartitionByExpression(expressions, child, numPartitions) => - exchange.ShuffleExchangeExec(HashPartitioning( - expressions, numPartitions), planLater(child)) :: Nil + case r: logical.RepartitionByExpression => + exchange.ShuffleExchangeExec(r.partitioning, planLater(r.child)) :: Nil case ExternalRDD(outputObjAttr, rdd) => ExternalRDDScanExec(outputObjAttr, rdd) :: Nil case r: LogicalRDD => RDDScanExec(r.output, r.rdd, "ExistingRDD", r.outputPartitioning, r.outputOrdering) :: Nil diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 72a5cc98fbec3..5e4c1a6a484fb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -358,6 +358,63 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { testData.select('key).collect().toSeq) } + test("repartition with SortOrder") { + // passing SortOrder expressions to .repartition() should result in an informative error + + def checkSortOrderErrorMsg[T](data: => Dataset[T]): Unit = { + val ex = intercept[IllegalArgumentException](data) + assert(ex.getMessage.contains("repartitionByRange")) + } + + checkSortOrderErrorMsg { + Seq(0).toDF("a").repartition(2, $"a".asc) + } + + checkSortOrderErrorMsg { + Seq((0, 0)).toDF("a", "b").repartition(2, $"a".asc, $"b") + } + } + + test("repartitionByRange") { + val data1d = Random.shuffle(0.to(9)) + val data2d = data1d.map(i => (i, data1d.size - i)) + + checkAnswer( + data1d.toDF("val").repartitionByRange(data1d.size, $"val".asc) + .select(spark_partition_id().as("id"), $"val"), + data1d.map(i => Row(i, i))) + + checkAnswer( + data1d.toDF("val").repartitionByRange(data1d.size, $"val".desc) + .select(spark_partition_id().as("id"), $"val"), + data1d.map(i => Row(i, data1d.size - 1 - i))) + + checkAnswer( + data1d.toDF("val").repartitionByRange(data1d.size, lit(42)) + .select(spark_partition_id().as("id"), $"val"), + data1d.map(i => Row(0, i))) + + checkAnswer( + data1d.toDF("val").repartitionByRange(data1d.size, lit(null), $"val".asc, rand()) + .select(spark_partition_id().as("id"), $"val"), + data1d.map(i => Row(i, i))) + + // .repartitionByRange() assumes .asc by default if no explicit sort order is specified + checkAnswer( + data2d.toDF("a", "b").repartitionByRange(data2d.size, $"a".desc, $"b") + .select(spark_partition_id().as("id"), $"a", $"b"), + data2d.toDF("a", "b").repartitionByRange(data2d.size, $"a".desc, $"b".asc) + .select(spark_partition_id().as("id"), $"a", $"b")) + + // at least one partition-by expression must be specified + intercept[IllegalArgumentException] { + data1d.toDF("val").repartitionByRange(data1d.size) + } + intercept[IllegalArgumentException] { + data1d.toDF("val").repartitionByRange(data1d.size, Seq.empty: _*) + } + } + test("coalesce") { intercept[IllegalArgumentException] { testData.select('key).coalesce(0) From 7e5f669eb684629c88218f8ec26c01a41a6fef32 Mon Sep 17 00:00:00 2001 From: gaborgsomogyi Date: Thu, 30 Nov 2017 19:20:32 -0600 Subject: [PATCH 05/37] =?UTF-8?q?[SPARK-22428][DOC]=20Add=20spark=20applic?= =?UTF-8?q?ation=20garbage=20collector=20configurat=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? The spark properties for configuring the ContextCleaner are not documented in the official documentation at https://spark.apache.org/docs/latest/configuration.html#available-properties. This PR adds the doc. ## How was this patch tested? Manual. ``` cd docs jekyll build open _site/configuration.html ``` Author: gaborgsomogyi Closes #19826 from gaborgsomogyi/SPARK-22428. --- docs/configuration.md | 40 ++++++++++++++++++++++++++++++++++++++++ 1 file changed, 40 insertions(+) diff --git a/docs/configuration.md b/docs/configuration.md index e42f866c40566..ef061dd39dcba 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1132,6 +1132,46 @@ Apart from these, the following properties are also available, and may be useful to get the replication level of the block to the initial number. + + spark.cleaner.periodicGC.interval + 30min + + Controls how often to trigger a garbage collection.

+ This context cleaner triggers cleanups only when weak references are garbage collected. + In long-running applications with large driver JVMs, where there is little memory pressure + on the driver, this may happen very occasionally or not at all. Not cleaning at all may + lead to executors running out of disk space after a while. + + + + spark.cleaner.referenceTracking + true + + Enables or disables context cleaning. + + + + spark.cleaner.referenceTracking.blocking + true + + Controls whether the cleaning thread should block on cleanup tasks (other than shuffle, which is controlled by + spark.cleaner.referenceTracking.blocking.shuffle Spark property). + + + + spark.cleaner.referenceTracking.blocking.shuffle + false + + Controls whether the cleaning thread should block on shuffle cleanup tasks. + + + + spark.cleaner.referenceTracking.cleanCheckpoints + false + + Controls whether to clean checkpoint files if the reference is out of scope. + + ### Execution Behavior From 7da1f5708cc96c18ddb3acd09542621275e71d83 Mon Sep 17 00:00:00 2001 From: Min Shen Date: Thu, 30 Nov 2017 19:24:44 -0600 Subject: [PATCH 06/37] =?UTF-8?q?[SPARK-22373]=20Bump=20Janino=20dependenc?= =?UTF-8?q?y=20version=20to=20fix=20thread=20safety=20issue=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit … with Janino when compiling generated code. ## What changes were proposed in this pull request? Bump up Janino dependency version to fix thread safety issue during compiling generated code ## How was this patch tested? Check https://issues.apache.org/jira/browse/SPARK-22373 for details. Converted part of the code in CodeGenerator into a standalone application, so the issue can be consistently reproduced locally. Verified that changing Janino dependency version resolved this issue. Author: Min Shen Closes #19839 from Victsm/SPARK-22373. --- dev/deps/spark-deps-hadoop-2.6 | 4 ++-- dev/deps/spark-deps-hadoop-2.7 | 4 ++-- pom.xml | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index 50ac6d139bbd4..8f508219c2ded 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -35,7 +35,7 @@ commons-beanutils-core-1.8.0.jar commons-cli-1.2.jar commons-codec-1.10.jar commons-collections-3.2.2.jar -commons-compiler-3.0.0.jar +commons-compiler-3.0.7.jar commons-compress-1.4.1.jar commons-configuration-1.6.jar commons-crypto-1.0.0.jar @@ -96,7 +96,7 @@ jackson-mapper-asl-1.9.13.jar jackson-module-paranamer-2.7.9.jar jackson-module-scala_2.11-2.6.7.1.jar jackson-xc-1.9.13.jar -janino-3.0.0.jar +janino-3.0.7.jar java-xmlbuilder-1.0.jar javassist-3.18.1-GA.jar javax.annotation-api-1.2.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 1b1e3166d53db..68e937f50b391 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -35,7 +35,7 @@ commons-beanutils-core-1.8.0.jar commons-cli-1.2.jar commons-codec-1.10.jar commons-collections-3.2.2.jar -commons-compiler-3.0.0.jar +commons-compiler-3.0.7.jar commons-compress-1.4.1.jar commons-configuration-1.6.jar commons-crypto-1.0.0.jar @@ -96,7 +96,7 @@ jackson-mapper-asl-1.9.13.jar jackson-module-paranamer-2.7.9.jar jackson-module-scala_2.11-2.6.7.1.jar jackson-xc-1.9.13.jar -janino-3.0.0.jar +janino-3.0.7.jar java-xmlbuilder-1.0.jar javassist-3.18.1-GA.jar javax.annotation-api-1.2.jar diff --git a/pom.xml b/pom.xml index 7bc66e7d19540..731ee86439eff 100644 --- a/pom.xml +++ b/pom.xml @@ -170,7 +170,7 @@ 3.5 3.2.10 - 3.0.0 + 3.0.7 2.22.2 2.9.3 3.5.2 From dc365422bb337d19ef39739c7c3cf9e53ec85d09 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Fri, 1 Dec 2017 10:53:16 +0800 Subject: [PATCH 07/37] =?UTF-8?q?[SPARK-22653]=20executorAddress=20registe?= =?UTF-8?q?red=20in=20CoarseGrainedSchedulerBac=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit https://issues.apache.org/jira/browse/SPARK-22653 executorRef.address can be null, pass the executorAddress which accounts for it being null a few lines above the fix. Manually tested this patch. You can reproduce the issue by running a simple spark-shell in yarn client mode with dynamic allocation and request some executors up front. Let those executors idle timeout. Get a heap dump. Without this fix, you will see that addressToExecutorId still contains the ids, with the fix addressToExecutorId is properly cleaned up. Author: Thomas Graves Closes #19850 from tgravescs/SPARK-22653. --- .../spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 22d9c4cf81c55..7bfb4d53c1834 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 @@ -182,7 +182,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp addressToExecutorId(executorAddress) = executorId totalCoreCount.addAndGet(cores) totalRegisteredExecutors.addAndGet(1) - val data = new ExecutorData(executorRef, executorRef.address, hostname, + val data = new ExecutorData(executorRef, executorAddress, hostname, cores, cores, logUrls) // This must be synchronized because variables mutated // in this block are read when requesting executors From 16adaf634bcca3074b448d95e72177eefdf50069 Mon Sep 17 00:00:00 2001 From: sujith71955 Date: Thu, 30 Nov 2017 20:45:30 -0800 Subject: [PATCH 08/37] [SPARK-22601][SQL] Data load is getting displayed successful on providing non existing nonlocal file path ## What changes were proposed in this pull request? When user tries to load data with a non existing hdfs file path system is not validating it and the load command operation is getting successful. This is misleading to the user. already there is a validation in the scenario of none existing local file path. This PR has added validation in the scenario of nonexisting hdfs file path ## How was this patch tested? UT has been added for verifying the issue, also snapshots has been added after the verification in a spark yarn cluster Author: sujith71955 Closes #19823 from sujith71955/master_LoadComand_Issue. --- .../org/apache/spark/sql/execution/command/tables.scala | 9 ++++++++- .../apache/spark/sql/hive/execution/HiveDDLSuite.scala | 9 +++++++++ 2 files changed, 17 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index c9f6e571ddab3..c42e6c3257fad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -340,7 +340,7 @@ case class LoadDataCommand( uri } else { val uri = new URI(path) - if (uri.getScheme() != null && uri.getAuthority() != null) { + val hdfsUri = if (uri.getScheme() != null && uri.getAuthority() != null) { uri } else { // Follow Hive's behavior: @@ -380,6 +380,13 @@ case class LoadDataCommand( } new URI(scheme, authority, absolutePath, uri.getQuery(), uri.getFragment()) } + val hadoopConf = sparkSession.sessionState.newHadoopConf() + val srcPath = new Path(hdfsUri) + val fs = srcPath.getFileSystem(hadoopConf) + if (!fs.exists(srcPath)) { + throw new AnalysisException(s"LOAD DATA input path does not exist: $path") + } + hdfsUri } if (partition.nonEmpty) { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 9063ef066aa84..6c11905ba8904 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -2141,4 +2141,13 @@ class HiveDDLSuite } } } + + test("load command for non local invalid path validation") { + withTable("tbl") { + sql("CREATE TABLE tbl(i INT, j STRING)") + val e = intercept[AnalysisException]( + sql("load data inpath '/doesnotexist.csv' into table tbl")) + assert(e.message.contains("LOAD DATA input path does not exist")) + } + } } From 9d06a9e0cf05af99ba210fabae1e77eccfce7986 Mon Sep 17 00:00:00 2001 From: Mark Petruska Date: Fri, 1 Dec 2017 05:14:12 -0600 Subject: [PATCH 09/37] [SPARK-22393][SPARK-SHELL] spark-shell can't find imported types in class constructors, extends clause ## What changes were proposed in this pull request? [SPARK-22393](https://issues.apache.org/jira/browse/SPARK-22393) ## How was this patch tested? With a new test case in `RepSuite` ---- This code is a retrofit of the Scala [SI-9881](https://github.com/scala/bug/issues/9881) bug fix, which never made it into the Scala 2.11 branches. Pushing these changes directly to the Scala repo is not practical (see: https://github.com/scala/scala/pull/6195). Author: Mark Petruska Closes #19846 from mpetruska/SPARK-22393. --- .../apache/spark/repl/SparkExprTyper.scala | 74 +++++++++++++ .../org/apache/spark/repl/SparkILoop.scala | 4 + .../spark/repl/SparkILoopInterpreter.scala | 103 ++++++++++++++++++ .../org/apache/spark/repl/ReplSuite.scala | 10 ++ 4 files changed, 191 insertions(+) create mode 100644 repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala create mode 100644 repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoopInterpreter.scala diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala new file mode 100644 index 0000000000000..724ce9af49f77 --- /dev/null +++ b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala @@ -0,0 +1,74 @@ +/* + * 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.repl + +import scala.tools.nsc.interpreter.{ExprTyper, IR} + +trait SparkExprTyper extends ExprTyper { + + import repl._ + import global.{reporter => _, Import => _, _} + import naming.freshInternalVarName + + def doInterpret(code: String): IR.Result = { + // interpret/interpretSynthetic may change the phase, + // which would have unintended effects on types. + val savedPhase = phase + try interpretSynthetic(code) finally phase = savedPhase + } + + override def symbolOfLine(code: String): Symbol = { + def asExpr(): Symbol = { + val name = freshInternalVarName() + // Typing it with a lazy val would give us the right type, but runs + // into compiler bugs with things like existentials, so we compile it + // behind a def and strip the NullaryMethodType which wraps the expr. + val line = "def " + name + " = " + code + + doInterpret(line) match { + case IR.Success => + val sym0 = symbolOfTerm(name) + // drop NullaryMethodType + sym0.cloneSymbol setInfo exitingTyper(sym0.tpe_*.finalResultType) + case _ => NoSymbol + } + } + + def asDefn(): Symbol = { + val old = repl.definedSymbolList.toSet + + doInterpret(code) match { + case IR.Success => + repl.definedSymbolList filterNot old match { + case Nil => NoSymbol + case sym :: Nil => sym + case syms => NoSymbol.newOverloaded(NoPrefix, syms) + } + case _ => NoSymbol + } + } + + def asError(): Symbol = { + doInterpret(code) + NoSymbol + } + + beSilentDuring(asExpr()) orElse beSilentDuring(asDefn()) orElse asError() + } + +} diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala index 3ce7cc7c85f74..e69441a475e9a 100644 --- a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -35,6 +35,10 @@ class SparkILoop(in0: Option[BufferedReader], out: JPrintWriter) def this(in0: BufferedReader, out: JPrintWriter) = this(Some(in0), out) def this() = this(None, new JPrintWriter(Console.out, true)) + override def createInterpreter(): Unit = { + intp = new SparkILoopInterpreter(settings, out) + } + val initializationCommands: Seq[String] = Seq( """ @transient val spark = if (org.apache.spark.repl.Main.sparkSession != null) { diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoopInterpreter.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoopInterpreter.scala new file mode 100644 index 0000000000000..0803426403af5 --- /dev/null +++ b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoopInterpreter.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.repl + +import scala.tools.nsc.Settings +import scala.tools.nsc.interpreter._ + +class SparkILoopInterpreter(settings: Settings, out: JPrintWriter) extends IMain(settings, out) { + self => + + override lazy val memberHandlers = new { + val intp: self.type = self + } with MemberHandlers { + import intp.global._ + + override def chooseHandler(member: intp.global.Tree): MemberHandler = member match { + case member: Import => new SparkImportHandler(member) + case _ => super.chooseHandler (member) + } + + class SparkImportHandler(imp: Import) extends ImportHandler(imp: Import) { + + override def targetType: Type = intp.global.rootMirror.getModuleIfDefined("" + expr) match { + case NoSymbol => intp.typeOfExpression("" + expr) + case sym => sym.tpe + } + + private def safeIndexOf(name: Name, s: String): Int = fixIndexOf(name, pos(name, s)) + private def fixIndexOf(name: Name, idx: Int): Int = if (idx == name.length) -1 else idx + private def pos(name: Name, s: String): Int = { + var i = name.pos(s.charAt(0), 0) + val sLen = s.length() + if (sLen == 1) return i + while (i + sLen <= name.length) { + var j = 1 + while (s.charAt(j) == name.charAt(i + j)) { + j += 1 + if (j == sLen) return i + } + i = name.pos(s.charAt(0), i + 1) + } + name.length + } + + private def isFlattenedSymbol(sym: Symbol): Boolean = + sym.owner.isPackageClass && + sym.name.containsName(nme.NAME_JOIN_STRING) && + sym.owner.info.member(sym.name.take( + safeIndexOf(sym.name, nme.NAME_JOIN_STRING))) != NoSymbol + + private def importableTargetMembers = + importableMembers(exitingTyper(targetType)).filterNot(isFlattenedSymbol).toList + + def isIndividualImport(s: ImportSelector): Boolean = + s.name != nme.WILDCARD && s.rename != nme.WILDCARD + def isWildcardImport(s: ImportSelector): Boolean = + s.name == nme.WILDCARD + + // non-wildcard imports + private def individualSelectors = selectors filter isIndividualImport + + override val importsWildcard: Boolean = selectors exists isWildcardImport + + lazy val importableSymbolsWithRenames: List[(Symbol, Name)] = { + val selectorRenameMap = + individualSelectors.flatMap(x => x.name.bothNames zip x.rename.bothNames).toMap + importableTargetMembers flatMap (m => selectorRenameMap.get(m.name) map (m -> _)) + } + + override lazy val individualSymbols: List[Symbol] = importableSymbolsWithRenames map (_._1) + override lazy val wildcardSymbols: List[Symbol] = + if (importsWildcard) importableTargetMembers else Nil + + } + + } + + object expressionTyper extends { + val repl: SparkILoopInterpreter.this.type = self + } with SparkExprTyper { } + + override def symbolOfLine(code: String): global.Symbol = + expressionTyper.symbolOfLine(code) + + override def typeOfExpression(expr: String, silent: Boolean): global.Type = + expressionTyper.typeOfExpression(expr, silent) + +} diff --git a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala index 905b41cdc1594..a5053521f8e31 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -227,4 +227,14 @@ class ReplSuite extends SparkFunSuite { assertDoesNotContain("error: not found: value sc", output) } + test("spark-shell should find imported types in class constructors and extends clause") { + val output = runInterpreter("local", + """ + |import org.apache.spark.Partition + |class P(p: Partition) + |class P(val index: Int) extends Partition + """.stripMargin) + assertDoesNotContain("error: not found: type Partition", output) + } + } From ee10ca7ec6cf7fbaab3f95a097b46936d97d0835 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Fri, 1 Dec 2017 13:02:03 -0800 Subject: [PATCH 10/37] [SPARK-22638][SS] Use a separate queue for StreamingQueryListenerBus ## What changes were proposed in this pull request? Use a separate Spark event queue for StreamingQueryListenerBus so that if there are many non-streaming events, streaming query listeners don't need to wait for other Spark listeners and can catch up. ## How was this patch tested? Jenkins Author: Shixiong Zhu Closes #19838 from zsxwing/SPARK-22638. --- .../scala/org/apache/spark/scheduler/LiveListenerBus.scala | 4 +++- .../sql/execution/streaming/StreamingQueryListenerBus.scala | 6 +++++- 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala index 2f93c497c5771..23121402b1025 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala @@ -87,7 +87,9 @@ private[spark] class LiveListenerBus(conf: SparkConf) { * of each other (each one uses a separate thread for delivering events), allowing slower * listeners to be somewhat isolated from others. */ - private def addToQueue(listener: SparkListenerInterface, queue: String): Unit = synchronized { + private[spark] def addToQueue( + listener: SparkListenerInterface, + queue: String): Unit = synchronized { if (stopped.get()) { throw new IllegalStateException("LiveListenerBus is stopped.") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryListenerBus.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryListenerBus.scala index 07e39023c8366..7dd491ede9d05 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryListenerBus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryListenerBus.scala @@ -40,7 +40,7 @@ class StreamingQueryListenerBus(sparkListenerBus: LiveListenerBus) import StreamingQueryListener._ - sparkListenerBus.addToSharedQueue(this) + sparkListenerBus.addToQueue(this, StreamingQueryListenerBus.STREAM_EVENT_QUERY) /** * RunIds of active queries whose events are supposed to be forwarded by this ListenerBus @@ -130,3 +130,7 @@ class StreamingQueryListenerBus(sparkListenerBus: LiveListenerBus) } } } + +object StreamingQueryListenerBus { + val STREAM_EVENT_QUERY = "streams" +} From aa4cf2b19e4cf5588af7e2192e0e9f687cd84bc5 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Sat, 2 Dec 2017 11:55:43 +0900 Subject: [PATCH 11/37] [SPARK-22651][PYTHON][ML] Prevent initiating multiple Hive clients for ImageSchema.readImages ## What changes were proposed in this pull request? Calling `ImageSchema.readImages` multiple times as below in PySpark shell: ```python from pyspark.ml.image import ImageSchema data_path = 'data/mllib/images/kittens' _ = ImageSchema.readImages(data_path, recursive=True, dropImageFailures=True).collect() _ = ImageSchema.readImages(data_path, recursive=True, dropImageFailures=True).collect() ``` throws an error as below: ``` ... org.datanucleus.exceptions.NucleusDataStoreException: Unable to open a test connection to the given database. JDBC url = jdbc:derby:;databaseName=metastore_db;create=true, username = APP. Terminating connection pool (set lazyInit to true if you expect to start your database after your app). Original Exception: ------ java.sql.SQLException: Failed to start database 'metastore_db' with class loader org.apache.spark.sql.hive.client.IsolatedClientLoader$$anon$1742f639f, see the next exception for details. ... at org.apache.derby.jdbc.AutoloadedDriver.connect(Unknown Source) ... at org.apache.hadoop.hive.metastore.HiveMetaStore.newRetryingHMSHandler(HiveMetaStore.java:5762) ... at org.apache.spark.sql.hive.client.HiveClientImpl.newState(HiveClientImpl.scala:180) ... at org.apache.spark.sql.hive.HiveExternalCatalog$$anonfun$databaseExists$1.apply$mcZ$sp(HiveExternalCatalog.scala:195) at org.apache.spark.sql.hive.HiveExternalCatalog$$anonfun$databaseExists$1.apply(HiveExternalCatalog.scala:195) at org.apache.spark.sql.hive.HiveExternalCatalog$$anonfun$databaseExists$1.apply(HiveExternalCatalog.scala:195) at org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:97) at org.apache.spark.sql.hive.HiveExternalCatalog.databaseExists(HiveExternalCatalog.scala:194) at org.apache.spark.sql.internal.SharedState.externalCatalog$lzycompute(SharedState.scala:100) at org.apache.spark.sql.internal.SharedState.externalCatalog(SharedState.scala:88) at org.apache.spark.sql.hive.HiveSessionStateBuilder.externalCatalog(HiveSessionStateBuilder.scala:39) at org.apache.spark.sql.hive.HiveSessionStateBuilder.catalog$lzycompute(HiveSessionStateBuilder.scala:54) at org.apache.spark.sql.hive.HiveSessionStateBuilder.catalog(HiveSessionStateBuilder.scala:52) at org.apache.spark.sql.hive.HiveSessionStateBuilder$$anon$1.(HiveSessionStateBuilder.scala:69) at org.apache.spark.sql.hive.HiveSessionStateBuilder.analyzer(HiveSessionStateBuilder.scala:69) at org.apache.spark.sql.internal.BaseSessionStateBuilder$$anonfun$build$2.apply(BaseSessionStateBuilder.scala:293) at org.apache.spark.sql.internal.BaseSessionStateBuilder$$anonfun$build$2.apply(BaseSessionStateBuilder.scala:293) at org.apache.spark.sql.internal.SessionState.analyzer$lzycompute(SessionState.scala:79) at org.apache.spark.sql.internal.SessionState.analyzer(SessionState.scala:79) at org.apache.spark.sql.execution.QueryExecution.analyzed$lzycompute(QueryExecution.scala:70) at org.apache.spark.sql.execution.QueryExecution.analyzed(QueryExecution.scala:68) at org.apache.spark.sql.execution.QueryExecution.assertAnalyzed(QueryExecution.scala:51) at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:70) at org.apache.spark.sql.SparkSession.internalCreateDataFrame(SparkSession.scala:574) at org.apache.spark.sql.SparkSession.createDataFrame(SparkSession.scala:593) at org.apache.spark.sql.SparkSession.createDataFrame(SparkSession.scala:348) at org.apache.spark.sql.SparkSession.createDataFrame(SparkSession.scala:348) at org.apache.spark.ml.image.ImageSchema$$anonfun$readImages$2$$anonfun$apply$1.apply(ImageSchema.scala:253) ... Caused by: ERROR XJ040: Failed to start database 'metastore_db' with class loader org.apache.spark.sql.hive.client.IsolatedClientLoader$$anon$1742f639f, see the next exception for details. at org.apache.derby.iapi.error.StandardException.newException(Unknown Source) at org.apache.derby.impl.jdbc.SQLExceptionFactory.wrapArgsForTransportAcrossDRDA(Unknown Source) ... 121 more Caused by: ERROR XSDB6: Another instance of Derby may have already booted the database /.../spark/metastore_db. ... Traceback (most recent call last): File "", line 1, in File "/.../spark/python/pyspark/ml/image.py", line 190, in readImages dropImageFailures, float(sampleRatio), seed) File "/.../spark/python/lib/py4j-0.10.6-src.zip/py4j/java_gateway.py", line 1160, in __call__ File "/.../spark/python/pyspark/sql/utils.py", line 69, in deco raise AnalysisException(s.split(': ', 1)[1], stackTrace) pyspark.sql.utils.AnalysisException: u'java.lang.RuntimeException: java.lang.RuntimeException: Unable to instantiate org.apache.hadoop.hive.ql.metadata.SessionHiveMetaStoreClient;' ``` Seems we better stick to `SparkSession.builder.getOrCreate()` like: https://github.com/apache/spark/blob/51620e288b5e0a7fffc3899c9deadabace28e6d7/python/pyspark/sql/streaming.py#L329 https://github.com/apache/spark/blob/dc5d34d8dcd6526d1dfdac8606661561c7576a62/python/pyspark/sql/column.py#L541 https://github.com/apache/spark/blob/33d43bf1b6f55594187066f0e38ba3985fa2542b/python/pyspark/sql/readwriter.py#L105 ## How was this patch tested? This was tested as below in PySpark shell: ```python from pyspark.ml.image import ImageSchema data_path = 'data/mllib/images/kittens' _ = ImageSchema.readImages(data_path, recursive=True, dropImageFailures=True).collect() _ = ImageSchema.readImages(data_path, recursive=True, dropImageFailures=True).collect() ``` Author: hyukjinkwon Closes #19845 from HyukjinKwon/SPARK-22651. --- python/pyspark/ml/image.py | 5 ++--- python/pyspark/ml/tests.py | 31 ++++++++++++++++++++++++++++++- 2 files changed, 32 insertions(+), 4 deletions(-) diff --git a/python/pyspark/ml/image.py b/python/pyspark/ml/image.py index 2b61aa9c0d9e9..384599dc0c532 100644 --- a/python/pyspark/ml/image.py +++ b/python/pyspark/ml/image.py @@ -201,9 +201,8 @@ def readImages(self, path, recursive=False, numPartitions=-1, .. versionadded:: 2.3.0 """ - ctx = SparkContext._active_spark_context - spark = SparkSession(ctx) - image_schema = ctx._jvm.org.apache.spark.ml.image.ImageSchema + spark = SparkSession.builder.getOrCreate() + image_schema = spark._jvm.org.apache.spark.ml.image.ImageSchema jsession = spark._jsparkSession jresult = image_schema.readImages(path, jsession, recursive, numPartitions, dropImageFailures, float(sampleRatio), seed) diff --git a/python/pyspark/ml/tests.py b/python/pyspark/ml/tests.py index 89ef555cf3442..3a0b816c367ec 100755 --- a/python/pyspark/ml/tests.py +++ b/python/pyspark/ml/tests.py @@ -67,7 +67,7 @@ from pyspark.ml.util import * from pyspark.ml.wrapper import JavaParams, JavaWrapper from pyspark.serializers import PickleSerializer -from pyspark.sql import DataFrame, Row, SparkSession +from pyspark.sql import DataFrame, Row, SparkSession, HiveContext from pyspark.sql.functions import rand from pyspark.sql.types import DoubleType, IntegerType from pyspark.storagelevel import * @@ -1855,6 +1855,35 @@ def test_read_images(self): lambda: ImageSchema.toImage("a")) +class ImageReaderTest2(PySparkTestCase): + + @classmethod + def setUpClass(cls): + PySparkTestCase.setUpClass() + # Note that here we enable Hive's support. + try: + cls.sc._jvm.org.apache.hadoop.hive.conf.HiveConf() + except py4j.protocol.Py4JError: + cls.tearDownClass() + raise unittest.SkipTest("Hive is not available") + except TypeError: + cls.tearDownClass() + raise unittest.SkipTest("Hive is not available") + cls.spark = HiveContext._createForTesting(cls.sc) + + @classmethod + def tearDownClass(cls): + PySparkTestCase.tearDownClass() + cls.spark.sparkSession.stop() + + def test_read_images_multiple_times(self): + # This test case is to check if `ImageSchema.readImages` tries to + # initiate Hive client multiple times. See SPARK-22651. + data_path = 'data/mllib/images/kittens' + ImageSchema.readImages(data_path, recursive=True, dropImageFailures=True) + ImageSchema.readImages(data_path, recursive=True, dropImageFailures=True) + + class ALSTest(SparkSessionTestCase): def test_storage_levels(self): From d2cf95aa63f5f5c9423f0455c2bfbee7833c9982 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sat, 2 Dec 2017 07:37:02 -0600 Subject: [PATCH 12/37] [SPARK-22634][BUILD] Update Bouncy Castle to 1.58 ## What changes were proposed in this pull request? Update Bouncy Castle to 1.58, and jets3t to 0.9.4 to (sort of) match. ## How was this patch tested? Existing tests Author: Sean Owen Closes #19859 from srowen/SPARK-22634. --- dev/deps/spark-deps-hadoop-2.6 | 8 +++----- dev/deps/spark-deps-hadoop-2.7 | 8 +++----- pom.xml | 8 +++++++- 3 files changed, 13 insertions(+), 11 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index 8f508219c2ded..2c68b73095c4d 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -21,7 +21,7 @@ avro-1.7.7.jar avro-ipc-1.7.7.jar avro-mapred-1.7.7-hadoop2.jar base64-2.3.8.jar -bcprov-jdk15on-1.51.jar +bcprov-jdk15on-1.58.jar bonecp-0.8.0.RELEASE.jar breeze-macros_2.11-0.13.2.jar breeze_2.11-0.13.2.jar @@ -97,7 +97,7 @@ jackson-module-paranamer-2.7.9.jar jackson-module-scala_2.11-2.6.7.1.jar jackson-xc-1.9.13.jar janino-3.0.7.jar -java-xmlbuilder-1.0.jar +java-xmlbuilder-1.1.jar javassist-3.18.1-GA.jar javax.annotation-api-1.2.jar javax.inject-1.jar @@ -115,7 +115,7 @@ jersey-container-servlet-core-2.22.2.jar jersey-guava-2.22.2.jar jersey-media-jaxb-2.22.2.jar jersey-server-2.22.2.jar -jets3t-0.9.3.jar +jets3t-0.9.4.jar jetty-6.1.26.jar jetty-util-6.1.26.jar jline-2.12.1.jar @@ -137,14 +137,12 @@ log4j-1.2.17.jar lz4-java-1.4.0.jar machinist_2.11-0.6.1.jar macro-compat_2.11-1.1.1.jar -mail-1.4.7.jar mesos-1.4.0-shaded-protobuf.jar metrics-core-3.1.5.jar metrics-graphite-3.1.5.jar metrics-json-3.1.5.jar metrics-jvm-3.1.5.jar minlog-1.3.0.jar -mx4j-3.0.2.jar netty-3.9.9.Final.jar netty-all-4.0.47.Final.jar objenesis-2.1.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 68e937f50b391..2aaac600b3ec3 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -21,7 +21,7 @@ avro-1.7.7.jar avro-ipc-1.7.7.jar avro-mapred-1.7.7-hadoop2.jar base64-2.3.8.jar -bcprov-jdk15on-1.51.jar +bcprov-jdk15on-1.58.jar bonecp-0.8.0.RELEASE.jar breeze-macros_2.11-0.13.2.jar breeze_2.11-0.13.2.jar @@ -97,7 +97,7 @@ jackson-module-paranamer-2.7.9.jar jackson-module-scala_2.11-2.6.7.1.jar jackson-xc-1.9.13.jar janino-3.0.7.jar -java-xmlbuilder-1.0.jar +java-xmlbuilder-1.1.jar javassist-3.18.1-GA.jar javax.annotation-api-1.2.jar javax.inject-1.jar @@ -115,7 +115,7 @@ jersey-container-servlet-core-2.22.2.jar jersey-guava-2.22.2.jar jersey-media-jaxb-2.22.2.jar jersey-server-2.22.2.jar -jets3t-0.9.3.jar +jets3t-0.9.4.jar jetty-6.1.26.jar jetty-util-6.1.26.jar jline-2.12.1.jar @@ -138,14 +138,12 @@ log4j-1.2.17.jar lz4-java-1.4.0.jar machinist_2.11-0.6.1.jar macro-compat_2.11-1.1.1.jar -mail-1.4.7.jar mesos-1.4.0-shaded-protobuf.jar metrics-core-3.1.5.jar metrics-graphite-3.1.5.jar metrics-json-3.1.5.jar metrics-jvm-3.1.5.jar minlog-1.3.0.jar -mx4j-3.0.2.jar netty-3.9.9.Final.jar netty-all-4.0.47.Final.jar objenesis-2.1.jar diff --git a/pom.xml b/pom.xml index 731ee86439eff..07bca9d267da0 100644 --- a/pom.xml +++ b/pom.xml @@ -141,7 +141,7 @@ 3.1.5 1.7.7 hadoop2 - 0.9.3 + 0.9.4 1.7.3 1.11.76 @@ -985,6 +985,12 @@ + + org.bouncycastle + bcprov-jdk15on + + 1.58 + org.apache.hadoop hadoop-yarn-api From f23dddf105aef88531b3572ad70889cf2fc300c9 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 3 Dec 2017 22:21:44 +0800 Subject: [PATCH 13/37] [SPARK-20682][SPARK-15474][SPARK-21791] Add new ORCFileFormat based on ORC 1.4.1 ## What changes were proposed in this pull request? Since [SPARK-2883](https://issues.apache.org/jira/browse/SPARK-2883), Apache Spark supports Apache ORC inside `sql/hive` module with Hive dependency. This PR aims to add a new ORC data source inside `sql/core` and to replace the old ORC data source eventually. This PR resolves the following three issues. - [SPARK-20682](https://issues.apache.org/jira/browse/SPARK-20682): Add new ORCFileFormat based on Apache ORC 1.4.1 - [SPARK-15474](https://issues.apache.org/jira/browse/SPARK-15474): ORC data source fails to write and read back empty dataframe - [SPARK-21791](https://issues.apache.org/jira/browse/SPARK-21791): ORC should support column names with dot ## How was this patch tested? Pass the Jenkins with the existing all tests and new tests for SPARK-15474 and SPARK-21791. Author: Dongjoon Hyun Author: Wenchen Fan Closes #19651 from dongjoon-hyun/SPARK-20682. --- .../datasources/orc/OrcDeserializer.scala | 243 ++++++++++++++++++ .../datasources/orc/OrcFileFormat.scala | 139 +++++++++- .../datasources/orc/OrcFilters.scala | 210 +++++++++++++++ .../datasources/orc/OrcOutputWriter.scala | 53 ++++ .../datasources/orc/OrcSerializer.scala | 228 ++++++++++++++++ .../execution/datasources/orc/OrcUtils.scala | 113 ++++++++ .../org/apache/spark/sql/SQLQuerySuite.scala | 25 ++ 7 files changed, 1009 insertions(+), 2 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcDeserializer.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcOutputWriter.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcSerializer.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcDeserializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcDeserializer.scala new file mode 100644 index 0000000000000..4ecc54bd2fd96 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcDeserializer.scala @@ -0,0 +1,243 @@ +/* + * 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.sql.execution.datasources.orc + +import org.apache.hadoop.io._ +import org.apache.orc.mapred.{OrcList, OrcMap, OrcStruct, OrcTimestamp} +import org.apache.orc.storage.serde2.io.{DateWritable, HiveDecimalWritable} + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{SpecificInternalRow, UnsafeArrayData} +import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String + +/** + * A deserializer to deserialize ORC structs to Spark rows. + */ +class OrcDeserializer( + dataSchema: StructType, + requiredSchema: StructType, + requestedColIds: Array[Int]) { + + private val resultRow = new SpecificInternalRow(requiredSchema.map(_.dataType)) + + private val fieldWriters: Array[WritableComparable[_] => Unit] = { + requiredSchema.zipWithIndex + // The value of missing columns are always null, do not need writers. + .filterNot { case (_, index) => requestedColIds(index) == -1 } + .map { case (f, index) => + val writer = newWriter(f.dataType, new RowUpdater(resultRow)) + (value: WritableComparable[_]) => writer(index, value) + }.toArray + } + + private val validColIds = requestedColIds.filterNot(_ == -1) + + def deserialize(orcStruct: OrcStruct): InternalRow = { + var i = 0 + while (i < validColIds.length) { + val value = orcStruct.getFieldValue(validColIds(i)) + if (value == null) { + resultRow.setNullAt(i) + } else { + fieldWriters(i)(value) + } + i += 1 + } + resultRow + } + + /** + * Creates a writer to write ORC values to Catalyst data structure at the given ordinal. + */ + private def newWriter( + dataType: DataType, updater: CatalystDataUpdater): (Int, WritableComparable[_]) => Unit = + dataType match { + case NullType => (ordinal, _) => + updater.setNullAt(ordinal) + + case BooleanType => (ordinal, value) => + updater.setBoolean(ordinal, value.asInstanceOf[BooleanWritable].get) + + case ByteType => (ordinal, value) => + updater.setByte(ordinal, value.asInstanceOf[ByteWritable].get) + + case ShortType => (ordinal, value) => + updater.setShort(ordinal, value.asInstanceOf[ShortWritable].get) + + case IntegerType => (ordinal, value) => + updater.setInt(ordinal, value.asInstanceOf[IntWritable].get) + + case LongType => (ordinal, value) => + updater.setLong(ordinal, value.asInstanceOf[LongWritable].get) + + case FloatType => (ordinal, value) => + updater.setFloat(ordinal, value.asInstanceOf[FloatWritable].get) + + case DoubleType => (ordinal, value) => + updater.setDouble(ordinal, value.asInstanceOf[DoubleWritable].get) + + case StringType => (ordinal, value) => + updater.set(ordinal, UTF8String.fromBytes(value.asInstanceOf[Text].copyBytes)) + + case BinaryType => (ordinal, value) => + val binary = value.asInstanceOf[BytesWritable] + val bytes = new Array[Byte](binary.getLength) + System.arraycopy(binary.getBytes, 0, bytes, 0, binary.getLength) + updater.set(ordinal, bytes) + + case DateType => (ordinal, value) => + updater.setInt(ordinal, DateTimeUtils.fromJavaDate(value.asInstanceOf[DateWritable].get)) + + case TimestampType => (ordinal, value) => + updater.setLong(ordinal, DateTimeUtils.fromJavaTimestamp(value.asInstanceOf[OrcTimestamp])) + + case DecimalType.Fixed(precision, scale) => (ordinal, value) => + val decimal = value.asInstanceOf[HiveDecimalWritable].getHiveDecimal() + val v = Decimal(decimal.bigDecimalValue, decimal.precision(), decimal.scale()) + v.changePrecision(precision, scale) + updater.set(ordinal, v) + + case st: StructType => (ordinal, value) => + val result = new SpecificInternalRow(st) + val fieldUpdater = new RowUpdater(result) + val fieldConverters = st.map(_.dataType).map { dt => + newWriter(dt, fieldUpdater) + }.toArray + val orcStruct = value.asInstanceOf[OrcStruct] + + var i = 0 + while (i < st.length) { + val value = orcStruct.getFieldValue(i) + if (value == null) { + result.setNullAt(i) + } else { + fieldConverters(i)(i, value) + } + i += 1 + } + + updater.set(ordinal, result) + + case ArrayType(elementType, _) => (ordinal, value) => + val orcArray = value.asInstanceOf[OrcList[WritableComparable[_]]] + val length = orcArray.size() + val result = createArrayData(elementType, length) + val elementUpdater = new ArrayDataUpdater(result) + val elementConverter = newWriter(elementType, elementUpdater) + + var i = 0 + while (i < length) { + val value = orcArray.get(i) + if (value == null) { + result.setNullAt(i) + } else { + elementConverter(i, value) + } + i += 1 + } + + updater.set(ordinal, result) + + case MapType(keyType, valueType, _) => (ordinal, value) => + val orcMap = value.asInstanceOf[OrcMap[WritableComparable[_], WritableComparable[_]]] + val length = orcMap.size() + val keyArray = createArrayData(keyType, length) + val keyUpdater = new ArrayDataUpdater(keyArray) + val keyConverter = newWriter(keyType, keyUpdater) + val valueArray = createArrayData(valueType, length) + val valueUpdater = new ArrayDataUpdater(valueArray) + val valueConverter = newWriter(valueType, valueUpdater) + + var i = 0 + val it = orcMap.entrySet().iterator() + while (it.hasNext) { + val entry = it.next() + keyConverter(i, entry.getKey) + val value = entry.getValue + if (value == null) { + valueArray.setNullAt(i) + } else { + valueConverter(i, value) + } + i += 1 + } + + updater.set(ordinal, new ArrayBasedMapData(keyArray, valueArray)) + + case udt: UserDefinedType[_] => newWriter(udt.sqlType, updater) + + case _ => + throw new UnsupportedOperationException(s"$dataType is not supported yet.") + } + + private def createArrayData(elementType: DataType, length: Int): ArrayData = elementType match { + case BooleanType => UnsafeArrayData.fromPrimitiveArray(new Array[Boolean](length)) + case ByteType => UnsafeArrayData.fromPrimitiveArray(new Array[Byte](length)) + case ShortType => UnsafeArrayData.fromPrimitiveArray(new Array[Short](length)) + case IntegerType => UnsafeArrayData.fromPrimitiveArray(new Array[Int](length)) + case LongType => UnsafeArrayData.fromPrimitiveArray(new Array[Long](length)) + case FloatType => UnsafeArrayData.fromPrimitiveArray(new Array[Float](length)) + case DoubleType => UnsafeArrayData.fromPrimitiveArray(new Array[Double](length)) + case _ => new GenericArrayData(new Array[Any](length)) + } + + /** + * A base interface for updating values inside catalyst data structure like `InternalRow` and + * `ArrayData`. + */ + sealed trait CatalystDataUpdater { + def set(ordinal: Int, value: Any): Unit + + def setNullAt(ordinal: Int): Unit = set(ordinal, null) + def setBoolean(ordinal: Int, value: Boolean): Unit = set(ordinal, value) + def setByte(ordinal: Int, value: Byte): Unit = set(ordinal, value) + def setShort(ordinal: Int, value: Short): Unit = set(ordinal, value) + def setInt(ordinal: Int, value: Int): Unit = set(ordinal, value) + def setLong(ordinal: Int, value: Long): Unit = set(ordinal, value) + def setDouble(ordinal: Int, value: Double): Unit = set(ordinal, value) + def setFloat(ordinal: Int, value: Float): Unit = set(ordinal, value) + } + + final class RowUpdater(row: InternalRow) extends CatalystDataUpdater { + override def setNullAt(ordinal: Int): Unit = row.setNullAt(ordinal) + override def set(ordinal: Int, value: Any): Unit = row.update(ordinal, value) + + override def setBoolean(ordinal: Int, value: Boolean): Unit = row.setBoolean(ordinal, value) + override def setByte(ordinal: Int, value: Byte): Unit = row.setByte(ordinal, value) + override def setShort(ordinal: Int, value: Short): Unit = row.setShort(ordinal, value) + override def setInt(ordinal: Int, value: Int): Unit = row.setInt(ordinal, value) + override def setLong(ordinal: Int, value: Long): Unit = row.setLong(ordinal, value) + override def setDouble(ordinal: Int, value: Double): Unit = row.setDouble(ordinal, value) + override def setFloat(ordinal: Int, value: Float): Unit = row.setFloat(ordinal, value) + } + + final class ArrayDataUpdater(array: ArrayData) extends CatalystDataUpdater { + override def setNullAt(ordinal: Int): Unit = array.setNullAt(ordinal) + override def set(ordinal: Int, value: Any): Unit = array.update(ordinal, value) + + override def setBoolean(ordinal: Int, value: Boolean): Unit = array.setBoolean(ordinal, value) + override def setByte(ordinal: Int, value: Byte): Unit = array.setByte(ordinal, value) + override def setShort(ordinal: Int, value: Short): Unit = array.setShort(ordinal, value) + override def setInt(ordinal: Int, value: Int): Unit = array.setInt(ordinal, value) + override def setLong(ordinal: Int, value: Long): Unit = array.setLong(ordinal, value) + override def setDouble(ordinal: Int, value: Double): Unit = array.setDouble(ordinal, value) + override def setFloat(ordinal: Int, value: Float): Unit = array.setFloat(ordinal, value) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala index 215740e90fe84..75c42213db3c8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala @@ -17,10 +17,29 @@ package org.apache.spark.sql.execution.datasources.orc -import org.apache.orc.TypeDescription +import java.io._ +import java.net.URI +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hadoop.mapred.JobConf +import org.apache.hadoop.mapreduce._ +import org.apache.hadoop.mapreduce.lib.input.FileSplit +import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl +import org.apache.orc._ +import org.apache.orc.OrcConf.{COMPRESS, MAPRED_OUTPUT_SCHEMA} +import org.apache.orc.mapred.OrcStruct +import org.apache.orc.mapreduce._ + +import org.apache.spark.TaskContext import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.sources._ +import org.apache.spark.sql.types._ +import org.apache.spark.util.SerializableConfiguration private[sql] object OrcFileFormat { private def checkFieldName(name: String): Unit = { @@ -39,3 +58,119 @@ private[sql] object OrcFileFormat { names.foreach(checkFieldName) } } + +/** + * New ORC File Format based on Apache ORC. + */ +class OrcFileFormat + extends FileFormat + with DataSourceRegister + with Serializable { + + override def shortName(): String = "orc" + + override def toString: String = "ORC" + + override def hashCode(): Int = getClass.hashCode() + + override def equals(other: Any): Boolean = other.isInstanceOf[OrcFileFormat] + + override def inferSchema( + sparkSession: SparkSession, + options: Map[String, String], + files: Seq[FileStatus]): Option[StructType] = { + OrcUtils.readSchema(sparkSession, files) + } + + override def prepareWrite( + sparkSession: SparkSession, + job: Job, + options: Map[String, String], + dataSchema: StructType): OutputWriterFactory = { + val orcOptions = new OrcOptions(options, sparkSession.sessionState.conf) + + val conf = job.getConfiguration + + conf.set(MAPRED_OUTPUT_SCHEMA.getAttribute, dataSchema.catalogString) + + conf.set(COMPRESS.getAttribute, orcOptions.compressionCodec) + + conf.asInstanceOf[JobConf] + .setOutputFormat(classOf[org.apache.orc.mapred.OrcOutputFormat[OrcStruct]]) + + new OutputWriterFactory { + override def newInstance( + path: String, + dataSchema: StructType, + context: TaskAttemptContext): OutputWriter = { + new OrcOutputWriter(path, dataSchema, context) + } + + override def getFileExtension(context: TaskAttemptContext): String = { + val compressionExtension: String = { + val name = context.getConfiguration.get(COMPRESS.getAttribute) + OrcUtils.extensionsForCompressionCodecNames.getOrElse(name, "") + } + + compressionExtension + ".orc" + } + } + } + + override def isSplitable( + sparkSession: SparkSession, + options: Map[String, String], + path: Path): Boolean = { + true + } + + override def buildReader( + sparkSession: SparkSession, + dataSchema: StructType, + partitionSchema: StructType, + requiredSchema: StructType, + filters: Seq[Filter], + options: Map[String, String], + hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = { + if (sparkSession.sessionState.conf.orcFilterPushDown) { + OrcFilters.createFilter(dataSchema, filters).foreach { f => + OrcInputFormat.setSearchArgument(hadoopConf, f, dataSchema.fieldNames) + } + } + + val broadcastedConf = + sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) + val isCaseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis + + (file: PartitionedFile) => { + val conf = broadcastedConf.value.value + + val requestedColIdsOrEmptyFile = OrcUtils.requestedColumnIds( + isCaseSensitive, dataSchema, requiredSchema, new Path(new URI(file.filePath)), conf) + + if (requestedColIdsOrEmptyFile.isEmpty) { + Iterator.empty + } else { + val requestedColIds = requestedColIdsOrEmptyFile.get + assert(requestedColIds.length == requiredSchema.length, + "[BUG] requested column IDs do not match required schema") + conf.set(OrcConf.INCLUDE_COLUMNS.getAttribute, + requestedColIds.filter(_ != -1).sorted.mkString(",")) + + val fileSplit = + new FileSplit(new Path(new URI(file.filePath)), file.start, file.length, Array.empty) + val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) + val taskAttemptContext = new TaskAttemptContextImpl(conf, attemptId) + + val orcRecordReader = new OrcInputFormat[OrcStruct] + .createRecordReader(fileSplit, taskAttemptContext) + val iter = new RecordReaderIterator[OrcStruct](orcRecordReader) + Option(TaskContext.get()).foreach(_.addTaskCompletionListener(_ => iter.close())) + + val unsafeProjection = UnsafeProjection.create(requiredSchema) + val deserializer = new OrcDeserializer(dataSchema, requiredSchema, requestedColIds) + iter.map(value => unsafeProjection(deserializer.deserialize(value))) + } + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala new file mode 100644 index 0000000000000..cec256cc1b498 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -0,0 +1,210 @@ +/* + * 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.sql.execution.datasources.orc + +import org.apache.orc.storage.ql.io.sarg.{PredicateLeaf, SearchArgument, SearchArgumentFactory} +import org.apache.orc.storage.ql.io.sarg.SearchArgument.Builder +import org.apache.orc.storage.serde2.io.HiveDecimalWritable + +import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.types._ + +/** + * Helper object for building ORC `SearchArgument`s, which are used for ORC predicate push-down. + * + * Due to limitation of ORC `SearchArgument` builder, we had to end up with a pretty weird double- + * checking pattern when converting `And`/`Or`/`Not` filters. + * + * An ORC `SearchArgument` must be built in one pass using a single builder. For example, you can't + * build `a = 1` and `b = 2` first, and then combine them into `a = 1 AND b = 2`. This is quite + * different from the cases in Spark SQL or Parquet, where complex filters can be easily built using + * existing simpler ones. + * + * The annoying part is that, `SearchArgument` builder methods like `startAnd()`, `startOr()`, and + * `startNot()` mutate internal state of the builder instance. This forces us to translate all + * convertible filters with a single builder instance. However, before actually converting a filter, + * we've no idea whether it can be recognized by ORC or not. Thus, when an inconvertible filter is + * found, we may already end up with a builder whose internal state is inconsistent. + * + * For example, to convert an `And` filter with builder `b`, we call `b.startAnd()` first, and then + * try to convert its children. Say we convert `left` child successfully, but find that `right` + * child is inconvertible. Alas, `b.startAnd()` call can't be rolled back, and `b` is inconsistent + * now. + * + * The workaround employed here is that, for `And`/`Or`/`Not`, we first try to convert their + * children with brand new builders, and only do the actual conversion with the right builder + * instance when the children are proven to be convertible. + * + * P.S.: Hive seems to use `SearchArgument` together with `ExprNodeGenericFuncDesc` only. Usage of + * builder methods mentioned above can only be found in test code, where all tested filters are + * known to be convertible. + */ +private[orc] object OrcFilters { + + /** + * Create ORC filter as a SearchArgument instance. + */ + def createFilter(schema: StructType, filters: Seq[Filter]): Option[SearchArgument] = { + val dataTypeMap = schema.map(f => f.name -> f.dataType).toMap + + // First, tries to convert each filter individually to see whether it's convertible, and then + // collect all convertible ones to build the final `SearchArgument`. + val convertibleFilters = for { + filter <- filters + _ <- buildSearchArgument(dataTypeMap, filter, SearchArgumentFactory.newBuilder()) + } yield filter + + for { + // Combines all convertible filters using `And` to produce a single conjunction + conjunction <- convertibleFilters.reduceOption(org.apache.spark.sql.sources.And) + // Then tries to build a single ORC `SearchArgument` for the conjunction predicate + builder <- buildSearchArgument(dataTypeMap, conjunction, SearchArgumentFactory.newBuilder()) + } yield builder.build() + } + + /** + * Return true if this is a searchable type in ORC. + * Both CharType and VarcharType are cleaned at AstBuilder. + */ + private def isSearchableType(dataType: DataType) = dataType match { + // TODO: SPARK-21787 Support for pushing down filters for DateType in ORC + case BinaryType | DateType => false + case _: AtomicType => true + case _ => false + } + + /** + * Get PredicateLeafType which is corresponding to the given DataType. + */ + private def getPredicateLeafType(dataType: DataType) = dataType match { + case BooleanType => PredicateLeaf.Type.BOOLEAN + case ByteType | ShortType | IntegerType | LongType => PredicateLeaf.Type.LONG + case FloatType | DoubleType => PredicateLeaf.Type.FLOAT + case StringType => PredicateLeaf.Type.STRING + case DateType => PredicateLeaf.Type.DATE + case TimestampType => PredicateLeaf.Type.TIMESTAMP + case _: DecimalType => PredicateLeaf.Type.DECIMAL + case _ => throw new UnsupportedOperationException(s"DataType: $dataType") + } + + /** + * Cast literal values for filters. + * + * We need to cast to long because ORC raises exceptions + * at 'checkLiteralType' of SearchArgumentImpl.java. + */ + private def castLiteralValue(value: Any, dataType: DataType): Any = dataType match { + case ByteType | ShortType | IntegerType | LongType => + value.asInstanceOf[Number].longValue + case FloatType | DoubleType => + value.asInstanceOf[Number].doubleValue() + case _: DecimalType => + val decimal = value.asInstanceOf[java.math.BigDecimal] + val decimalWritable = new HiveDecimalWritable(decimal.longValue) + decimalWritable.mutateEnforcePrecisionScale(decimal.precision, decimal.scale) + decimalWritable + case _ => value + } + + /** + * Build a SearchArgument and return the builder so far. + */ + private def buildSearchArgument( + dataTypeMap: Map[String, DataType], + expression: Filter, + builder: Builder): Option[Builder] = { + def newBuilder = SearchArgumentFactory.newBuilder() + + def getType(attribute: String): PredicateLeaf.Type = + getPredicateLeafType(dataTypeMap(attribute)) + + import org.apache.spark.sql.sources._ + + expression match { + case And(left, right) => + // At here, it is not safe to just convert one side if we do not understand the + // other side. Here is an example used to explain the reason. + // Let's say we have NOT(a = 2 AND b in ('1')) and we do not understand how to + // convert b in ('1'). If we only convert a = 2, we will end up with a filter + // NOT(a = 2), which will generate wrong results. + // Pushing one side of AND down is only safe to do at the top level. + // You can see ParquetRelation's initializeLocalJobFunc method as an example. + for { + _ <- buildSearchArgument(dataTypeMap, left, newBuilder) + _ <- buildSearchArgument(dataTypeMap, right, newBuilder) + lhs <- buildSearchArgument(dataTypeMap, left, builder.startAnd()) + rhs <- buildSearchArgument(dataTypeMap, right, lhs) + } yield rhs.end() + + case Or(left, right) => + for { + _ <- buildSearchArgument(dataTypeMap, left, newBuilder) + _ <- buildSearchArgument(dataTypeMap, right, newBuilder) + lhs <- buildSearchArgument(dataTypeMap, left, builder.startOr()) + rhs <- buildSearchArgument(dataTypeMap, right, lhs) + } yield rhs.end() + + case Not(child) => + for { + _ <- buildSearchArgument(dataTypeMap, child, newBuilder) + negate <- buildSearchArgument(dataTypeMap, child, builder.startNot()) + } yield negate.end() + + // NOTE: For all case branches dealing with leaf predicates below, the additional `startAnd()` + // call is mandatory. ORC `SearchArgument` builder requires that all leaf predicates must be + // wrapped by a "parent" predicate (`And`, `Or`, or `Not`). + + case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + val castedValue = castLiteralValue(value, dataTypeMap(attribute)) + Some(builder.startAnd().equals(attribute, getType(attribute), castedValue).end()) + + case EqualNullSafe(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + val castedValue = castLiteralValue(value, dataTypeMap(attribute)) + Some(builder.startAnd().nullSafeEquals(attribute, getType(attribute), castedValue).end()) + + case LessThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + val castedValue = castLiteralValue(value, dataTypeMap(attribute)) + Some(builder.startAnd().lessThan(attribute, getType(attribute), castedValue).end()) + + case LessThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + val castedValue = castLiteralValue(value, dataTypeMap(attribute)) + Some(builder.startAnd().lessThanEquals(attribute, getType(attribute), castedValue).end()) + + case GreaterThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + val castedValue = castLiteralValue(value, dataTypeMap(attribute)) + Some(builder.startNot().lessThanEquals(attribute, getType(attribute), castedValue).end()) + + case GreaterThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + val castedValue = castLiteralValue(value, dataTypeMap(attribute)) + Some(builder.startNot().lessThan(attribute, getType(attribute), castedValue).end()) + + case IsNull(attribute) if isSearchableType(dataTypeMap(attribute)) => + Some(builder.startAnd().isNull(attribute, getType(attribute)).end()) + + case IsNotNull(attribute) if isSearchableType(dataTypeMap(attribute)) => + Some(builder.startNot().isNull(attribute, getType(attribute)).end()) + + case In(attribute, values) if isSearchableType(dataTypeMap(attribute)) => + val castedValues = values.map(v => castLiteralValue(v, dataTypeMap(attribute))) + Some(builder.startAnd().in(attribute, getType(attribute), + castedValues.map(_.asInstanceOf[AnyRef]): _*).end()) + + case _ => None + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcOutputWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcOutputWriter.scala new file mode 100644 index 0000000000000..84755bfa301f0 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcOutputWriter.scala @@ -0,0 +1,53 @@ +/* + * 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.sql.execution.datasources.orc + +import org.apache.hadoop.fs.Path +import org.apache.hadoop.io.NullWritable +import org.apache.hadoop.mapreduce.TaskAttemptContext +import org.apache.orc.mapred.OrcStruct +import org.apache.orc.mapreduce.OrcOutputFormat + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.OutputWriter +import org.apache.spark.sql.types._ + +private[orc] class OrcOutputWriter( + path: String, + dataSchema: StructType, + context: TaskAttemptContext) + extends OutputWriter { + + private[this] val serializer = new OrcSerializer(dataSchema) + + private val recordWriter = { + new OrcOutputFormat[OrcStruct]() { + override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { + new Path(path) + } + }.getRecordWriter(context) + } + + override def write(row: InternalRow): Unit = { + recordWriter.write(NullWritable.get(), serializer.serialize(row)) + } + + override def close(): Unit = { + recordWriter.close(context) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcSerializer.scala new file mode 100644 index 0000000000000..899af0750cadf --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcSerializer.scala @@ -0,0 +1,228 @@ +/* + * 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.sql.execution.datasources.orc + +import org.apache.hadoop.io._ +import org.apache.orc.TypeDescription +import org.apache.orc.mapred.{OrcList, OrcMap, OrcStruct, OrcTimestamp} +import org.apache.orc.storage.common.`type`.HiveDecimal +import org.apache.orc.storage.serde2.io.{DateWritable, HiveDecimalWritable} + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.SpecializedGetters +import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.types._ + +/** + * A serializer to serialize Spark rows to ORC structs. + */ +class OrcSerializer(dataSchema: StructType) { + + private val result = createOrcValue(dataSchema).asInstanceOf[OrcStruct] + private val converters = dataSchema.map(_.dataType).map(newConverter(_)).toArray + + def serialize(row: InternalRow): OrcStruct = { + var i = 0 + while (i < converters.length) { + if (row.isNullAt(i)) { + result.setFieldValue(i, null) + } else { + result.setFieldValue(i, converters(i)(row, i)) + } + i += 1 + } + result + } + + private type Converter = (SpecializedGetters, Int) => WritableComparable[_] + + /** + * Creates a converter to convert Catalyst data at the given ordinal to ORC values. + */ + private def newConverter( + dataType: DataType, + reuseObj: Boolean = true): Converter = dataType match { + case NullType => (getter, ordinal) => null + + case BooleanType => + if (reuseObj) { + val result = new BooleanWritable() + (getter, ordinal) => + result.set(getter.getBoolean(ordinal)) + result + } else { + (getter, ordinal) => new BooleanWritable(getter.getBoolean(ordinal)) + } + + case ByteType => + if (reuseObj) { + val result = new ByteWritable() + (getter, ordinal) => + result.set(getter.getByte(ordinal)) + result + } else { + (getter, ordinal) => new ByteWritable(getter.getByte(ordinal)) + } + + case ShortType => + if (reuseObj) { + val result = new ShortWritable() + (getter, ordinal) => + result.set(getter.getShort(ordinal)) + result + } else { + (getter, ordinal) => new ShortWritable(getter.getShort(ordinal)) + } + + case IntegerType => + if (reuseObj) { + val result = new IntWritable() + (getter, ordinal) => + result.set(getter.getInt(ordinal)) + result + } else { + (getter, ordinal) => new IntWritable(getter.getInt(ordinal)) + } + + + case LongType => + if (reuseObj) { + val result = new LongWritable() + (getter, ordinal) => + result.set(getter.getLong(ordinal)) + result + } else { + (getter, ordinal) => new LongWritable(getter.getLong(ordinal)) + } + + case FloatType => + if (reuseObj) { + val result = new FloatWritable() + (getter, ordinal) => + result.set(getter.getFloat(ordinal)) + result + } else { + (getter, ordinal) => new FloatWritable(getter.getFloat(ordinal)) + } + + case DoubleType => + if (reuseObj) { + val result = new DoubleWritable() + (getter, ordinal) => + result.set(getter.getDouble(ordinal)) + result + } else { + (getter, ordinal) => new DoubleWritable(getter.getDouble(ordinal)) + } + + + // Don't reuse the result object for string and binary as it would cause extra data copy. + case StringType => (getter, ordinal) => + new Text(getter.getUTF8String(ordinal).getBytes) + + case BinaryType => (getter, ordinal) => + new BytesWritable(getter.getBinary(ordinal)) + + case DateType => + if (reuseObj) { + val result = new DateWritable() + (getter, ordinal) => + result.set(getter.getInt(ordinal)) + result + } else { + (getter, ordinal) => new DateWritable(getter.getInt(ordinal)) + } + + // The following cases are already expensive, reusing object or not doesn't matter. + + case TimestampType => (getter, ordinal) => + val ts = DateTimeUtils.toJavaTimestamp(getter.getLong(ordinal)) + val result = new OrcTimestamp(ts.getTime) + result.setNanos(ts.getNanos) + result + + case DecimalType.Fixed(precision, scale) => (getter, ordinal) => + val d = getter.getDecimal(ordinal, precision, scale) + new HiveDecimalWritable(HiveDecimal.create(d.toJavaBigDecimal)) + + case st: StructType => (getter, ordinal) => + val result = createOrcValue(st).asInstanceOf[OrcStruct] + val fieldConverters = st.map(_.dataType).map(newConverter(_)) + val numFields = st.length + val struct = getter.getStruct(ordinal, numFields) + var i = 0 + while (i < numFields) { + if (struct.isNullAt(i)) { + result.setFieldValue(i, null) + } else { + result.setFieldValue(i, fieldConverters(i)(struct, i)) + } + i += 1 + } + result + + case ArrayType(elementType, _) => (getter, ordinal) => + val result = createOrcValue(dataType).asInstanceOf[OrcList[WritableComparable[_]]] + // Need to put all converted values to a list, can't reuse object. + val elementConverter = newConverter(elementType, reuseObj = false) + val array = getter.getArray(ordinal) + var i = 0 + while (i < array.numElements()) { + if (array.isNullAt(i)) { + result.add(null) + } else { + result.add(elementConverter(array, i)) + } + i += 1 + } + result + + case MapType(keyType, valueType, _) => (getter, ordinal) => + val result = createOrcValue(dataType) + .asInstanceOf[OrcMap[WritableComparable[_], WritableComparable[_]]] + // Need to put all converted values to a list, can't reuse object. + val keyConverter = newConverter(keyType, reuseObj = false) + val valueConverter = newConverter(valueType, reuseObj = false) + val map = getter.getMap(ordinal) + val keyArray = map.keyArray() + val valueArray = map.valueArray() + var i = 0 + while (i < map.numElements()) { + val key = keyConverter(keyArray, i) + if (valueArray.isNullAt(i)) { + result.put(key, null) + } else { + result.put(key, valueConverter(valueArray, i)) + } + i += 1 + } + result + + case udt: UserDefinedType[_] => newConverter(udt.sqlType) + + case _ => + throw new UnsupportedOperationException(s"$dataType is not supported yet.") + } + + /** + * Return a Orc value object for the given Spark schema. + */ + private def createOrcValue(dataType: DataType) = { + OrcStruct.createValue(TypeDescription.fromString(dataType.catalogString)) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala new file mode 100644 index 0000000000000..b03ee06d04a16 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala @@ -0,0 +1,113 @@ +/* + * 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.sql.execution.datasources.orc + +import scala.collection.JavaConverters._ + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.orc.{OrcFile, TypeDescription} + +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.analysis.{caseInsensitiveResolution, caseSensitiveResolution} +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.sql.types._ + +object OrcUtils extends Logging { + + // The extensions for ORC compression codecs + val extensionsForCompressionCodecNames = Map( + "NONE" -> "", + "SNAPPY" -> ".snappy", + "ZLIB" -> ".zlib", + "LZO" -> ".lzo") + + def listOrcFiles(pathStr: String, conf: Configuration): Seq[Path] = { + val origPath = new Path(pathStr) + val fs = origPath.getFileSystem(conf) + val paths = SparkHadoopUtil.get.listLeafStatuses(fs, origPath) + .filterNot(_.isDirectory) + .map(_.getPath) + .filterNot(_.getName.startsWith("_")) + .filterNot(_.getName.startsWith(".")) + paths + } + + def readSchema(file: Path, conf: Configuration): Option[TypeDescription] = { + val fs = file.getFileSystem(conf) + val readerOptions = OrcFile.readerOptions(conf).filesystem(fs) + val reader = OrcFile.createReader(file, readerOptions) + val schema = reader.getSchema + if (schema.getFieldNames.size == 0) { + None + } else { + Some(schema) + } + } + + def readSchema(sparkSession: SparkSession, files: Seq[FileStatus]) + : Option[StructType] = { + val conf = sparkSession.sessionState.newHadoopConf() + // TODO: We need to support merge schema. Please see SPARK-11412. + files.map(_.getPath).flatMap(readSchema(_, conf)).headOption.map { schema => + logDebug(s"Reading schema from file $files, got Hive schema string: $schema") + CatalystSqlParser.parseDataType(schema.toString).asInstanceOf[StructType] + } + } + + /** + * Returns the requested column ids from the given ORC file. Column id can be -1, which means the + * requested column doesn't exist in the ORC file. Returns None if the given ORC file is empty. + */ + def requestedColumnIds( + isCaseSensitive: Boolean, + dataSchema: StructType, + requiredSchema: StructType, + file: Path, + conf: Configuration): Option[Array[Int]] = { + val fs = file.getFileSystem(conf) + val readerOptions = OrcFile.readerOptions(conf).filesystem(fs) + val reader = OrcFile.createReader(file, readerOptions) + val orcFieldNames = reader.getSchema.getFieldNames.asScala + if (orcFieldNames.isEmpty) { + // SPARK-8501: Some old empty ORC files always have an empty schema stored in their footer. + None + } else { + if (orcFieldNames.forall(_.startsWith("_col"))) { + // This is a ORC file written by Hive, no field names in the physical schema, assume the + // physical schema maps to the data scheme by index. + assert(orcFieldNames.length <= dataSchema.length, "The given data schema " + + s"${dataSchema.simpleString} has less fields than the actual ORC physical schema, " + + "no idea which columns were dropped, fail to read.") + Some(requiredSchema.fieldNames.map { name => + val index = dataSchema.fieldIndex(name) + if (index < orcFieldNames.length) { + index + } else { + -1 + } + }) + } else { + val resolver = if (isCaseSensitive) caseSensitiveResolution else caseInsensitiveResolution + Some(requiredSchema.fieldNames.map { name => orcFieldNames.indexWhere(resolver(_, name)) }) + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 5d0bba69daca1..31d9b909ad463 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -2757,4 +2757,29 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } } } + + // Only New OrcFileFormat supports this + Seq(classOf[org.apache.spark.sql.execution.datasources.orc.OrcFileFormat].getCanonicalName, + "parquet").foreach { format => + test(s"SPARK-15474 Write and read back non-emtpy schema with empty dataframe - $format") { + withTempPath { file => + val path = file.getCanonicalPath + val emptyDf = Seq((true, 1, "str")).toDF.limit(0) + emptyDf.write.format(format).save(path) + + val df = spark.read.format(format).load(path) + assert(df.schema.sameType(emptyDf.schema)) + checkAnswer(df, emptyDf) + } + } + } + + test("SPARK-21791 ORC should support column names with dot") { + val orc = classOf[org.apache.spark.sql.execution.datasources.orc.OrcFileFormat].getCanonicalName + withTempDir { dir => + val path = new File(dir, "orc").getCanonicalPath + Seq(Some(1), None).toDF("col.dots").write.format(orc).save(path) + assert(spark.read.format(orc).load(path).collect().length == 2) + } + } } From 2c16267f7ca392d717942a7654e90db60ba60770 Mon Sep 17 00:00:00 2001 From: Marco Gaido Date: Sun, 3 Dec 2017 22:56:03 +0800 Subject: [PATCH 14/37] [SPARK-22669][SQL] Avoid unnecessary function calls in code generation ## What changes were proposed in this pull request? In many parts of the codebase for code generation, we are splitting the code to avoid exceptions due to the 64KB method size limit. This is generating a lot of methods which are called every time, even though sometime this is not needed. As pointed out here: https://github.com/apache/spark/pull/19752#discussion_r153081547, this is a not negligible overhead which can be avoided. The PR applies the same approach used in #19752 also to the other places where this was feasible. ## How was this patch tested? existing UTs. Author: Marco Gaido Closes #19860 from mgaido91/SPARK-22669. --- .../expressions/nullExpressions.scala | 141 ++++++++++++------ .../sql/catalyst/expressions/predicates.scala | 68 ++++++--- 2 files changed, 140 insertions(+), 69 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala index 173e171910b69..3b52a0efd404a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala @@ -75,23 +75,51 @@ case class Coalesce(children: Seq[Expression]) extends Expression { ctx.addMutableState(ctx.JAVA_BOOLEAN, ev.isNull) ctx.addMutableState(ctx.javaType(dataType), ev.value) + // all the evals are meant to be in a do { ... } while (false); loop val evals = children.map { e => val eval = e.genCode(ctx) s""" - if (${ev.isNull}) { - ${eval.code} - if (!${eval.isNull}) { - ${ev.isNull} = false; - ${ev.value} = ${eval.value}; - } - } - """ + |${eval.code} + |if (!${eval.isNull}) { + | ${ev.isNull} = false; + | ${ev.value} = ${eval.value}; + | continue; + |} + """.stripMargin } + val code = if (ctx.INPUT_ROW == null || ctx.currentVars != null) { + evals.mkString("\n") + } else { + ctx.splitExpressions(evals, "coalesce", + ("InternalRow", ctx.INPUT_ROW) :: Nil, + makeSplitFunction = { + func => + s""" + |do { + | $func + |} while (false); + """.stripMargin + }, + foldFunctions = { funcCalls => + funcCalls.map { funcCall => + s""" + |$funcCall; + |if (!${ev.isNull}) { + | continue; + |} + """.stripMargin + }.mkString + }) + } - ev.copy(code = s""" - ${ev.isNull} = true; - ${ev.value} = ${ctx.defaultValue(dataType)}; - ${ctx.splitExpressions(evals)}""") + ev.copy(code = + s""" + |${ev.isNull} = true; + |${ev.value} = ${ctx.defaultValue(dataType)}; + |do { + | $code + |} while (false); + """.stripMargin) } } @@ -358,53 +386,70 @@ case class AtLeastNNonNulls(n: Int, children: Seq[Expression]) extends Predicate override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val nonnull = ctx.freshName("nonnull") + // all evals are meant to be inside a do { ... } while (false); loop val evals = children.map { e => val eval = e.genCode(ctx) e.dataType match { case DoubleType | FloatType => s""" - if ($nonnull < $n) { - ${eval.code} - if (!${eval.isNull} && !Double.isNaN(${eval.value})) { - $nonnull += 1; - } - } - """ + |if ($nonnull < $n) { + | ${eval.code} + | if (!${eval.isNull} && !Double.isNaN(${eval.value})) { + | $nonnull += 1; + | } + |} else { + | continue; + |} + """.stripMargin case _ => s""" - if ($nonnull < $n) { - ${eval.code} - if (!${eval.isNull}) { - $nonnull += 1; - } - } - """ + |if ($nonnull < $n) { + | ${eval.code} + | if (!${eval.isNull}) { + | $nonnull += 1; + | } + |} else { + | continue; + |} + """.stripMargin } } val code = if (ctx.INPUT_ROW == null || ctx.currentVars != null) { - evals.mkString("\n") - } else { - ctx.splitExpressions( - expressions = evals, - funcName = "atLeastNNonNulls", - arguments = ("InternalRow", ctx.INPUT_ROW) :: ("int", nonnull) :: Nil, - returnType = "int", - makeSplitFunction = { body => - s""" - $body - return $nonnull; - """ - }, - foldFunctions = { funcCalls => - funcCalls.map(funcCall => s"$nonnull = $funcCall;").mkString("\n") - } - ) - } + evals.mkString("\n") + } else { + ctx.splitExpressions( + expressions = evals, + funcName = "atLeastNNonNulls", + arguments = ("InternalRow", ctx.INPUT_ROW) :: (ctx.JAVA_INT, nonnull) :: Nil, + returnType = ctx.JAVA_INT, + makeSplitFunction = { body => + s""" + |do { + | $body + |} while (false); + |return $nonnull; + """.stripMargin + }, + foldFunctions = { funcCalls => + funcCalls.map(funcCall => + s""" + |$nonnull = $funcCall; + |if ($nonnull >= $n) { + | continue; + |} + """.stripMargin).mkString("\n") + } + ) + } - ev.copy(code = s""" - int $nonnull = 0; - $code - boolean ${ev.value} = $nonnull >= $n;""", isNull = "false") + ev.copy(code = + s""" + |${ctx.JAVA_INT} $nonnull = 0; + |do { + | $code + |} while (false); + |${ctx.JAVA_BOOLEAN} ${ev.value} = $nonnull >= $n; + """.stripMargin, isNull = "false") } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 1aaaaf1db48d1..75cc9b3bd8045 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -234,36 +234,62 @@ case class In(value: Expression, list: Seq[Expression]) extends Predicate { } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val javaDataType = ctx.javaType(value.dataType) val valueGen = value.genCode(ctx) val listGen = list.map(_.genCode(ctx)) ctx.addMutableState(ctx.JAVA_BOOLEAN, ev.value) ctx.addMutableState(ctx.JAVA_BOOLEAN, ev.isNull) val valueArg = ctx.freshName("valueArg") + // All the blocks are meant to be inside a do { ... } while (false); loop. + // The evaluation of variables can be stopped when we find a matching value. val listCode = listGen.map(x => s""" - if (!${ev.value}) { - ${x.code} - if (${x.isNull}) { - ${ev.isNull} = true; - } else if (${ctx.genEqual(value.dataType, valueArg, x.value)}) { - ${ev.isNull} = false; - ${ev.value} = true; + |${x.code} + |if (${x.isNull}) { + | ${ev.isNull} = true; + |} else if (${ctx.genEqual(value.dataType, valueArg, x.value)}) { + | ${ev.isNull} = false; + | ${ev.value} = true; + | continue; + |} + """.stripMargin) + val code = if (ctx.INPUT_ROW == null || ctx.currentVars != null) { + listCode.mkString("\n") + } else { + ctx.splitExpressions( + expressions = listCode, + funcName = "valueIn", + arguments = ("InternalRow", ctx.INPUT_ROW) :: (javaDataType, valueArg) :: Nil, + makeSplitFunction = { body => + s""" + |do { + | $body + |} while (false); + """.stripMargin + }, + foldFunctions = { funcCalls => + funcCalls.map(funcCall => + s""" + |$funcCall; + |if (${ev.value}) { + | continue; + |} + """.stripMargin).mkString("\n") } - } - """) - val listCodes = ctx.splitExpressions( - expressions = listCode, - funcName = "valueIn", - extraArguments = (ctx.javaType(value.dataType), valueArg) :: Nil) - ev.copy(code = s""" - ${valueGen.code} - ${ev.value} = false; - ${ev.isNull} = ${valueGen.isNull}; - if (!${ev.isNull}) { - ${ctx.javaType(value.dataType)} $valueArg = ${valueGen.value}; - $listCodes + ) } - """) + ev.copy(code = + s""" + |${valueGen.code} + |${ev.value} = false; + |${ev.isNull} = ${valueGen.isNull}; + |if (!${ev.isNull}) { + | $javaDataType $valueArg = ${valueGen.value}; + | do { + | $code + | } while (false); + |} + """.stripMargin) } override def sql: String = { From dff440f1ecdbce65cad377c44fd2abfd4eff9b44 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Sun, 3 Dec 2017 23:05:31 +0800 Subject: [PATCH 15/37] [SPARK-22626][SQL] deals with wrong Hive's statistics (zero rowCount) This pr to ensure that the Hive's statistics `totalSize` (or `rawDataSize`) > 0, `rowCount` also must be > 0. Otherwise may cause OOM when CBO is enabled. unit tests Author: Yuming Wang Closes #19831 from wangyum/SPARK-22626. --- .../sql/hive/client/HiveClientImpl.scala | 6 +++--- .../spark/sql/hive/StatisticsSuite.scala | 19 +++++++++++++++++++ 2 files changed, 22 insertions(+), 3 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index 47ce6ba83866c..77e836003b39f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -418,7 +418,7 @@ private[hive] class HiveClientImpl( // Note that this statistics could be overridden by Spark's statistics if that's available. val totalSize = properties.get(StatsSetupConst.TOTAL_SIZE).map(BigInt(_)) val rawDataSize = properties.get(StatsSetupConst.RAW_DATA_SIZE).map(BigInt(_)) - val rowCount = properties.get(StatsSetupConst.ROW_COUNT).map(BigInt(_)).filter(_ >= 0) + val rowCount = properties.get(StatsSetupConst.ROW_COUNT).map(BigInt(_)) // TODO: check if this estimate is valid for tables after partition pruning. // NOTE: getting `totalSize` directly from params is kind of hacky, but this should be // relatively cheap if parameters for the table are populated into the metastore. @@ -430,9 +430,9 @@ private[hive] class HiveClientImpl( // so when `totalSize` is zero, use `rawDataSize` instead. When `rawDataSize` is also zero, // return None. Later, we will use the other ways to estimate the statistics. if (totalSize.isDefined && totalSize.get > 0L) { - Some(CatalogStatistics(sizeInBytes = totalSize.get, rowCount = rowCount)) + Some(CatalogStatistics(sizeInBytes = totalSize.get, rowCount = rowCount.filter(_ > 0))) } else if (rawDataSize.isDefined && rawDataSize.get > 0) { - Some(CatalogStatistics(sizeInBytes = rawDataSize.get, rowCount = rowCount)) + Some(CatalogStatistics(sizeInBytes = rawDataSize.get, rowCount = rowCount.filter(_ > 0))) } else { // TODO: still fill the rowCount even if sizeInBytes is empty. Might break anything? None diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index 0cdd9305c6b6f..ee027e5308265 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -1360,4 +1360,23 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto } } + + test("Deals with wrong Hive's statistics (zero rowCount)") { + withTable("maybe_big") { + sql("CREATE TABLE maybe_big (c1 bigint)" + + "TBLPROPERTIES ('numRows'='0', 'rawDataSize'='60000000000', 'totalSize'='8000000000000')") + + val relation = spark.table("maybe_big").queryExecution.analyzed.children.head + .asInstanceOf[HiveTableRelation] + + val properties = relation.tableMeta.ignoredProperties + assert(properties("totalSize").toLong > 0) + assert(properties("rawDataSize").toLong > 0) + assert(properties("numRows").toLong == 0) + + assert(relation.stats.sizeInBytes > 0) + // May be cause OOM if rowCount == 0 when enables CBO, see SPARK-22626 for details. + assert(relation.stats.rowCount.isEmpty) + } + } } From 4131ad03f4d2dfcfb1e166e5dfdf0752479f7340 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Sun, 3 Dec 2017 23:52:37 -0800 Subject: [PATCH 16/37] [SPARK-22489][DOC][FOLLOWUP] Update broadcast behavior changes in migration section ## What changes were proposed in this pull request? Update broadcast behavior changes in migration section. ## How was this patch tested? N/A Author: Yuming Wang Closes #19858 from wangyum/SPARK-22489-migration. --- docs/sql-programming-guide.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index a1b9c3bbfd059..b76be9132dd03 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1776,6 +1776,8 @@ options. Note that, for DecimalType(38,0)*, the table above intentionally does not cover all other combinations of scales and precisions because currently we only infer decimal type like `BigInteger`/`BigInt`. For example, 1.1 is inferred as double type. - In PySpark, now we need Pandas 0.19.2 or upper if you want to use Pandas related functionalities, such as `toPandas`, `createDataFrame` from Pandas DataFrame, etc. - In PySpark, the behavior of timestamp values for Pandas related functionalities was changed to respect session timezone. If you want to use the old behavior, you need to set a configuration `spark.sql.execution.pandas.respectSessionTimeZone` to `False`. See [SPARK-22395](https://issues.apache.org/jira/browse/SPARK-22395) for details. + + - Since Spark 2.3, when either broadcast hash join or broadcast nested loop join is applicable, we prefer to broadcasting the table that is explicitly specified in a broadcast hint. For details, see the section [Broadcast Hint](#broadcast-hint-for-sql-queries) and [SPARK-22489](https://issues.apache.org/jira/browse/SPARK-22489). ## Upgrading From Spark SQL 2.1 to 2.2 From 3927bb9b460d2d944ecf3c8552d71e8a25d29655 Mon Sep 17 00:00:00 2001 From: Marco Gaido Date: Mon, 4 Dec 2017 11:07:27 -0600 Subject: [PATCH 17/37] [SPARK-22473][FOLLOWUP][TEST] Remove deprecated Date functions ## What changes were proposed in this pull request? #19696 replaced the deprecated usages for `Date` and `Waiter`, but a few methods were missed. The PR fixes the forgotten deprecated usages. ## How was this patch tested? existing UTs Author: Marco Gaido Closes #19875 from mgaido91/SPARK-22473_FOLLOWUP. --- .../spark/sql/catalyst/expressions/PredicateSuite.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala index 865092a659f26..0079e4e8d6f74 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala @@ -291,26 +291,26 @@ class PredicateSuite extends SparkFunSuite with ExpressionEvalHelper { private val udt = new ExamplePointUDT private val smallValues = - Seq(1.toByte, 1.toShort, 1, 1L, Decimal(1), Array(1.toByte), new Date(2000, 1, 1), + Seq(1.toByte, 1.toShort, 1, 1L, Decimal(1), Array(1.toByte), Date.valueOf("2000-01-01"), new Timestamp(1), "a", 1f, 1d, 0f, 0d, false, Array(1L, 2L)) .map(Literal(_)) ++ Seq(Literal.create(MyStruct(1L, "b")), Literal.create(MyStruct2(MyStruct(1L, "a"), Array(1, 1))), Literal.create(ArrayData.toArrayData(Array(1.0, 2.0)), udt)) private val largeValues = - Seq(2.toByte, 2.toShort, 2, 2L, Decimal(2), Array(2.toByte), new Date(2000, 1, 2), + Seq(2.toByte, 2.toShort, 2, 2L, Decimal(2), Array(2.toByte), Date.valueOf("2000-01-02"), new Timestamp(2), "b", 2f, 2d, Float.NaN, Double.NaN, true, Array(2L, 1L)) .map(Literal(_)) ++ Seq(Literal.create(MyStruct(2L, "b")), Literal.create(MyStruct2(MyStruct(1L, "a"), Array(1, 2))), Literal.create(ArrayData.toArrayData(Array(1.0, 3.0)), udt)) private val equalValues1 = - Seq(1.toByte, 1.toShort, 1, 1L, Decimal(1), Array(1.toByte), new Date(2000, 1, 1), + Seq(1.toByte, 1.toShort, 1, 1L, Decimal(1), Array(1.toByte), Date.valueOf("2000-01-01"), new Timestamp(1), "a", 1f, 1d, Float.NaN, Double.NaN, true, Array(1L, 2L)) .map(Literal(_)) ++ Seq(Literal.create(MyStruct(1L, "b")), Literal.create(MyStruct2(MyStruct(1L, "a"), Array(1, 1))), Literal.create(ArrayData.toArrayData(Array(1.0, 2.0)), udt)) private val equalValues2 = - Seq(1.toByte, 1.toShort, 1, 1L, Decimal(1), Array(1.toByte), new Date(2000, 1, 1), + Seq(1.toByte, 1.toShort, 1, 1L, Decimal(1), Array(1.toByte), Date.valueOf("2000-01-01"), new Timestamp(1), "a", 1f, 1d, Float.NaN, Double.NaN, true, Array(1L, 2L)) .map(Literal(_)) ++ Seq(Literal.create(MyStruct(1L, "b")), Literal.create(MyStruct2(MyStruct(1L, "a"), Array(1, 1))), From f81401e1cb39f2d6049b79dc8d61305f3371276f Mon Sep 17 00:00:00 2001 From: Reza Safi Date: Mon, 4 Dec 2017 09:23:48 -0800 Subject: [PATCH 18/37] [SPARK-22162] Executors and the driver should use consistent JobIDs in the RDD commit protocol I have modified SparkHadoopWriter so that executors and the driver always use consistent JobIds during the hadoop commit. Before SPARK-18191, spark always used the rddId, it just incorrectly named the variable stageId. After SPARK-18191, it used the rddId as the jobId on the driver's side, and the stageId as the jobId on the executors' side. With this change executors and the driver will consistently uses rddId as the jobId. Also with this change, during the hadoop commit protocol spark uses actual stageId to check whether a stage can be committed unlike before that it was using executors' jobId to do this check. In addition to the existing unit tests, a test has been added to check whether executors and the driver are using the same JobId. The test failed before this change and passed after applying this fix. Author: Reza Safi Closes #19848 from rezasafi/stagerddsimple. --- .../spark/internal/io/SparkHadoopWriter.scala | 12 ++--- .../spark/mapred/SparkHadoopMapRedUtil.scala | 5 ++- .../spark/rdd/PairRDDFunctionsSuite.scala | 44 +++++++++++++++++++ 3 files changed, 53 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala b/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala index 949d8c677998f..abf39213fa0d2 100644 --- a/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala +++ b/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala @@ -60,17 +60,17 @@ object SparkHadoopWriter extends Logging { config: HadoopWriteConfigUtil[K, V]): Unit = { // Extract context and configuration from RDD. val sparkContext = rdd.context - val stageId = rdd.id + val commitJobId = rdd.id // Set up a job. val jobTrackerId = createJobTrackerID(new Date()) - val jobContext = config.createJobContext(jobTrackerId, stageId) + val jobContext = config.createJobContext(jobTrackerId, commitJobId) config.initOutputFormat(jobContext) // Assert the output format/key/value class is set in JobConf. config.assertConf(jobContext, rdd.conf) - val committer = config.createCommitter(stageId) + val committer = config.createCommitter(commitJobId) committer.setupJob(jobContext) // Try to write all RDD partitions as a Hadoop OutputFormat. @@ -80,7 +80,7 @@ object SparkHadoopWriter extends Logging { context = context, config = config, jobTrackerId = jobTrackerId, - sparkStageId = context.stageId, + commitJobId = commitJobId, sparkPartitionId = context.partitionId, sparkAttemptNumber = context.attemptNumber, committer = committer, @@ -102,14 +102,14 @@ object SparkHadoopWriter extends Logging { context: TaskContext, config: HadoopWriteConfigUtil[K, V], jobTrackerId: String, - sparkStageId: Int, + commitJobId: Int, sparkPartitionId: Int, sparkAttemptNumber: Int, committer: FileCommitProtocol, iterator: Iterator[(K, V)]): TaskCommitMessage = { // Set up a task. val taskContext = config.createTaskAttemptContext( - jobTrackerId, sparkStageId, sparkPartitionId, sparkAttemptNumber) + jobTrackerId, commitJobId, sparkPartitionId, sparkAttemptNumber) committer.setupTask(taskContext) val (outputMetrics, callback) = initHadoopOutputMetrics(context) diff --git a/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala b/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala index 607283a306b8f..764735dc4eae7 100644 --- a/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala +++ b/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala @@ -70,7 +70,8 @@ object SparkHadoopMapRedUtil extends Logging { if (shouldCoordinateWithDriver) { val outputCommitCoordinator = SparkEnv.get.outputCommitCoordinator val taskAttemptNumber = TaskContext.get().attemptNumber() - val canCommit = outputCommitCoordinator.canCommit(jobId, splitId, taskAttemptNumber) + val stageId = TaskContext.get().stageId() + val canCommit = outputCommitCoordinator.canCommit(stageId, splitId, taskAttemptNumber) if (canCommit) { performCommit() @@ -80,7 +81,7 @@ object SparkHadoopMapRedUtil extends Logging { logInfo(message) // We need to abort the task so that the driver can reschedule new attempts, if necessary committer.abortTask(mrTaskContext) - throw new CommitDeniedException(message, jobId, splitId, taskAttemptNumber) + throw new CommitDeniedException(message, stageId, splitId, taskAttemptNumber) } } else { // Speculation is disabled or a user has chosen to manually bypass the commit coordination diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index 0a248b6064ee8..65d35264dc108 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -30,6 +30,7 @@ import org.apache.hadoop.mapreduce.{Job => NewJob, JobContext => NewJobContext, OutputCommitter => NewOutputCommitter, OutputFormat => NewOutputFormat, RecordWriter => NewRecordWriter, TaskAttemptContext => NewTaskAttempContext} import org.apache.hadoop.util.Progressable +import org.scalatest.Assertions import org.apache.spark._ import org.apache.spark.Partitioner @@ -524,6 +525,15 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext { pairs.saveAsNewAPIHadoopFile[ConfigTestFormat]("ignored") } + test("The JobId on the driver and executors should be the same during the commit") { + // Create more than one rdd to mimic stageId not equal to rddId + val pairs = sc.parallelize(Array((1, 2), (2, 3)), 2) + .map { p => (new Integer(p._1 + 1), new Integer(p._2 + 1)) } + .filter { p => p._1 > 0 } + pairs.saveAsNewAPIHadoopFile[YetAnotherFakeFormat]("ignored") + assert(JobID.jobid != -1) + } + test("saveAsHadoopFile should respect configured output committers") { val pairs = sc.parallelize(Array((new Integer(1), new Integer(1)))) val conf = new JobConf() @@ -908,6 +918,40 @@ class NewFakeFormatWithCallback() extends NewFakeFormat { } } +class YetAnotherFakeCommitter extends NewOutputCommitter with Assertions { + def setupJob(j: NewJobContext): Unit = { + JobID.jobid = j.getJobID().getId + } + + def needsTaskCommit(t: NewTaskAttempContext): Boolean = false + + def setupTask(t: NewTaskAttempContext): Unit = { + val jobId = t.getTaskAttemptID().getJobID().getId + assert(jobId === JobID.jobid) + } + + def commitTask(t: NewTaskAttempContext): Unit = {} + + def abortTask(t: NewTaskAttempContext): Unit = {} +} + +class YetAnotherFakeFormat() extends NewOutputFormat[Integer, Integer]() { + + def checkOutputSpecs(j: NewJobContext): Unit = {} + + def getRecordWriter(t: NewTaskAttempContext): NewRecordWriter[Integer, Integer] = { + new NewFakeWriter() + } + + def getOutputCommitter(t: NewTaskAttempContext): NewOutputCommitter = { + new YetAnotherFakeCommitter() + } +} + +object JobID { + var jobid = -1 +} + class ConfigTestFormat() extends NewFakeFormat() with Configurable { var setConfCalled = false From e1dd03e42c2131b167b1e80c761291e88bfdf03f Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 4 Dec 2017 11:05:03 -0800 Subject: [PATCH 19/37] [SPARK-22372][CORE, YARN] Make cluster submission use SparkApplication. The main goal of this change is to allow multiple cluster-mode submissions from the same JVM, without having them end up with mixed configuration. That is done by extending the SparkApplication trait, and doing so was reasonably trivial for standalone and mesos modes. For YARN mode, there was a complication. YARN used a "SPARK_YARN_MODE" system property to control behavior indirectly in a whole bunch of places, mainly in the SparkHadoopUtil / YarnSparkHadoopUtil classes. Most of the changes here are removing that. Since we removed support for Hadoop 1.x, some methods that lived in YarnSparkHadoopUtil can now live in SparkHadoopUtil. The remaining methods don't need to be part of the class, and can be called directly from the YarnSparkHadoopUtil object, so now there's a single implementation of SparkHadoopUtil. There were two places in the code that relied on SPARK_YARN_MODE to make decisions about YARN-specific functionality, and now explicitly check the master from the configuration for that instead: * fetching the external shuffle service port, which can come from the YARN configuration. * propagation of the authentication secret using Hadoop credentials. This also was cleaned up a little to not need so many methods in `SparkHadoopUtil`. With those out of the way, actually changing the YARN client to extend SparkApplication was easy. Tested with existing unit tests, and also by running YARN apps with auth and kerberos both on and off in a real cluster. Author: Marcelo Vanzin Closes #19631 from vanzin/SPARK-22372. --- .../org/apache/spark/SecurityManager.scala | 102 ++++++------- .../scala/org/apache/spark/SparkContext.scala | 3 - .../scala/org/apache/spark/SparkEnv.scala | 4 + .../org/apache/spark/deploy/Client.scala | 8 +- .../apache/spark/deploy/SparkHadoopUtil.scala | 48 +------ .../org/apache/spark/deploy/SparkSubmit.scala | 31 ++-- .../deploy/rest/RestSubmissionClient.scala | 31 ++-- .../CoarseGrainedExecutorBackend.scala | 10 +- .../scala/org/apache/spark/util/Utils.scala | 6 +- .../apache/spark/SecurityManagerSuite.scala | 31 +++- .../spark/deploy/SparkSubmitSuite.scala | 8 +- .../rest/StandaloneRestSubmitSuite.scala | 2 +- project/MimaExcludes.scala | 6 + .../spark/deploy/yarn/ApplicationMaster.scala | 54 +++---- .../org/apache/spark/deploy/yarn/Client.scala | 65 ++++----- .../spark/deploy/yarn/YarnRMClient.scala | 2 +- .../deploy/yarn/YarnSparkHadoopUtil.scala | 135 ++++++------------ .../yarn/security/AMCredentialRenewer.scala | 2 +- .../cluster/YarnClientSchedulerBackend.scala | 4 +- .../cluster/YarnClusterSchedulerBackend.scala | 2 +- .../deploy/yarn/BaseYarnClusterSuite.scala | 5 - .../spark/deploy/yarn/ClientSuite.scala | 31 +--- .../spark/deploy/yarn/YarnClusterSuite.scala | 20 ++- .../yarn/YarnSparkHadoopUtilSuite.scala | 49 +------ ...ARNHadoopDelegationTokenManagerSuite.scala | 11 +- 25 files changed, 274 insertions(+), 396 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index 2480e56b72ccf..4c1dbe3ffb4ad 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -19,6 +19,7 @@ package org.apache.spark import java.lang.{Byte => JByte} import java.net.{Authenticator, PasswordAuthentication} +import java.nio.charset.StandardCharsets.UTF_8 import java.security.{KeyStore, SecureRandom} import java.security.cert.X509Certificate import javax.net.ssl._ @@ -26,10 +27,11 @@ import javax.net.ssl._ import com.google.common.hash.HashCodes import com.google.common.io.Files import org.apache.hadoop.io.Text +import org.apache.hadoop.security.{Credentials, UserGroupInformation} -import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ +import org.apache.spark.launcher.SparkLauncher import org.apache.spark.network.sasl.SecretKeyHolder import org.apache.spark.util.Utils @@ -225,7 +227,6 @@ private[spark] class SecurityManager( setViewAclsGroups(sparkConf.get("spark.ui.view.acls.groups", "")); setModifyAclsGroups(sparkConf.get("spark.modify.acls.groups", "")); - private val secretKey = generateSecretKey() logInfo("SecurityManager: authentication " + (if (authOn) "enabled" else "disabled") + "; ui acls " + (if (aclsOn) "enabled" else "disabled") + "; users with view permissions: " + viewAcls.toString() + @@ -416,50 +417,6 @@ private[spark] class SecurityManager( def getIOEncryptionKey(): Option[Array[Byte]] = ioEncryptionKey - /** - * Generates or looks up the secret key. - * - * The way the key is stored depends on the Spark deployment mode. Yarn - * uses the Hadoop UGI. - * - * For non-Yarn deployments, If the config variable is not set - * we throw an exception. - */ - private def generateSecretKey(): String = { - if (!isAuthenticationEnabled) { - null - } else if (SparkHadoopUtil.get.isYarnMode) { - // In YARN mode, the secure cookie will be created by the driver and stashed in the - // user's credentials, where executors can get it. The check for an array of size 0 - // is because of the test code in YarnSparkHadoopUtilSuite. - val secretKey = SparkHadoopUtil.get.getSecretKeyFromUserCredentials(SECRET_LOOKUP_KEY) - if (secretKey == null || secretKey.length == 0) { - logDebug("generateSecretKey: yarn mode, secret key from credentials is null") - val rnd = new SecureRandom() - val length = sparkConf.getInt("spark.authenticate.secretBitLength", 256) / JByte.SIZE - val secret = new Array[Byte](length) - rnd.nextBytes(secret) - - val cookie = HashCodes.fromBytes(secret).toString() - SparkHadoopUtil.get.addSecretKeyToUserCredentials(SECRET_LOOKUP_KEY, cookie) - cookie - } else { - new Text(secretKey).toString - } - } else { - // user must have set spark.authenticate.secret config - // For Master/Worker, auth secret is in conf; for Executors, it is in env variable - Option(sparkConf.getenv(SecurityManager.ENV_AUTH_SECRET)) - .orElse(sparkConf.getOption(SecurityManager.SPARK_AUTH_SECRET_CONF)) match { - case Some(value) => value - case None => - throw new IllegalArgumentException( - "Error: a secret key must be specified via the " + - SecurityManager.SPARK_AUTH_SECRET_CONF + " config") - } - } - } - /** * Check to see if Acls for the UI are enabled * @return true if UI authentication is enabled, otherwise false @@ -542,7 +499,51 @@ private[spark] class SecurityManager( * Gets the secret key. * @return the secret key as a String if authentication is enabled, otherwise returns null */ - def getSecretKey(): String = secretKey + def getSecretKey(): String = { + if (isAuthenticationEnabled) { + val creds = UserGroupInformation.getCurrentUser().getCredentials() + Option(creds.getSecretKey(SECRET_LOOKUP_KEY)) + .map { bytes => new String(bytes, UTF_8) } + .orElse(Option(sparkConf.getenv(ENV_AUTH_SECRET))) + .orElse(sparkConf.getOption(SPARK_AUTH_SECRET_CONF)) + .getOrElse { + throw new IllegalArgumentException( + s"A secret key must be specified via the $SPARK_AUTH_SECRET_CONF config") + } + } else { + null + } + } + + /** + * Initialize the authentication secret. + * + * If authentication is disabled, do nothing. + * + * In YARN mode, generate a new secret and store it in the current user's credentials. + * + * In other modes, assert that the auth secret is set in the configuration. + */ + def initializeAuth(): Unit = { + if (!sparkConf.get(NETWORK_AUTH_ENABLED)) { + return + } + + if (sparkConf.get(SparkLauncher.SPARK_MASTER, null) != "yarn") { + require(sparkConf.contains(SPARK_AUTH_SECRET_CONF), + s"A secret key must be specified via the $SPARK_AUTH_SECRET_CONF config.") + return + } + + val rnd = new SecureRandom() + val length = sparkConf.getInt("spark.authenticate.secretBitLength", 256) / JByte.SIZE + val secretBytes = new Array[Byte](length) + rnd.nextBytes(secretBytes) + + val creds = new Credentials() + creds.addSecretKey(SECRET_LOOKUP_KEY, secretBytes) + UserGroupInformation.getCurrentUser().addCredentials(creds) + } // Default SecurityManager only has a single secret key, so ignore appId. override def getSaslUser(appId: String): String = getSaslUser() @@ -551,13 +552,12 @@ private[spark] class SecurityManager( private[spark] object SecurityManager { - val SPARK_AUTH_CONF: String = "spark.authenticate" - val SPARK_AUTH_SECRET_CONF: String = "spark.authenticate.secret" + val SPARK_AUTH_CONF = NETWORK_AUTH_ENABLED.key + val SPARK_AUTH_SECRET_CONF = "spark.authenticate.secret" // This is used to set auth secret to an executor's env variable. It should have the same // value as SPARK_AUTH_SECRET_CONF set in SparkConf val ENV_AUTH_SECRET = "_SPARK_AUTH_SECRET" // key used to store the spark secret in the Hadoop UGI - val SECRET_LOOKUP_KEY = "sparkCookie" - + val SECRET_LOOKUP_KEY = new Text("sparkCookie") } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index c174939ca2e54..71f1e7c7321bc 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -413,8 +413,6 @@ class SparkContext(config: SparkConf) extends Logging { } } - if (master == "yarn" && deployMode == "client") System.setProperty("SPARK_YARN_MODE", "true") - _listenerBus = new LiveListenerBus(_conf) // Initialize the app status store and listener before SparkEnv is created so that it gets @@ -1955,7 +1953,6 @@ class SparkContext(config: SparkConf) extends Logging { // `SparkContext` is stopped. localProperties.remove() // Unset YARN mode system env variable, to allow switching between cluster types. - System.clearProperty("SPARK_YARN_MODE") SparkContext.clearActiveContext() logInfo("Successfully stopped SparkContext") } diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 24928150315e8..72123f2232532 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -234,6 +234,10 @@ object SparkEnv extends Logging { } val securityManager = new SecurityManager(conf, ioEncryptionKey) + if (isDriver) { + securityManager.initializeAuth() + } + ioEncryptionKey.foreach { _ => if (!securityManager.isEncryptionEnabled()) { logWarning("I/O encryption enabled without RPC encryption: keys will be visible on the " + diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala index 7acb5c55bb252..d5145094ec079 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -217,8 +217,13 @@ object Client { println("Use ./bin/spark-submit with \"--master spark://host:port\"") } // scalastyle:on println + new ClientApp().start(args, new SparkConf()) + } +} - val conf = new SparkConf() +private[spark] class ClientApp extends SparkApplication { + + override def start(args: Array[String], conf: SparkConf): Unit = { val driverArgs = new ClientArguments(args) if (!conf.contains("spark.rpc.askTimeout")) { @@ -235,4 +240,5 @@ object Client { rpcEnv.awaitTermination() } + } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 17c7319b40f24..e14f9845e6db6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -75,9 +75,7 @@ class SparkHadoopUtil extends Logging { } def transferCredentials(source: UserGroupInformation, dest: UserGroupInformation) { - for (token <- source.getTokens.asScala) { - dest.addToken(token) - } + dest.addCredentials(source.getCredentials()) } /** @@ -120,16 +118,9 @@ class SparkHadoopUtil extends Logging { * Add any user credentials to the job conf which are necessary for running on a secure Hadoop * cluster. */ - def addCredentials(conf: JobConf) {} - - def isYarnMode(): Boolean = { false } - - def addSecretKeyToUserCredentials(key: String, secret: String) {} - - def getSecretKeyFromUserCredentials(key: String): Array[Byte] = { null } - - def getCurrentUserCredentials(): Credentials = { - UserGroupInformation.getCurrentUser().getCredentials() + def addCredentials(conf: JobConf): Unit = { + val jobCreds = conf.getCredentials() + jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials()) } def addCurrentUserCredentials(creds: Credentials): Unit = { @@ -328,17 +319,6 @@ class SparkHadoopUtil extends Logging { } } - /** - * Start a thread to periodically update the current user's credentials with new credentials so - * that access to secured service does not fail. - */ - private[spark] def startCredentialUpdater(conf: SparkConf) {} - - /** - * Stop the thread that does the credential updates. - */ - private[spark] def stopCredentialUpdater() {} - /** * Return a fresh Hadoop configuration, bypassing the HDFS cache mechanism. * This is to prevent the DFSClient from using an old cached token to connect to the NameNode. @@ -441,14 +421,7 @@ class SparkHadoopUtil extends Logging { object SparkHadoopUtil { - private lazy val hadoop = new SparkHadoopUtil - private lazy val yarn = try { - Utils.classForName("org.apache.spark.deploy.yarn.YarnSparkHadoopUtil") - .newInstance() - .asInstanceOf[SparkHadoopUtil] - } catch { - case e: Exception => throw new SparkException("Unable to load YARN support", e) - } + private lazy val instance = new SparkHadoopUtil val SPARK_YARN_CREDS_TEMP_EXTENSION = ".tmp" @@ -462,16 +435,7 @@ object SparkHadoopUtil { */ private[spark] val UPDATE_INPUT_METRICS_INTERVAL_RECORDS = 1000 - def get: SparkHadoopUtil = { - // Check each time to support changing to/from YARN - val yarnMode = java.lang.Boolean.parseBoolean( - System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE"))) - if (yarnMode) { - yarn - } else { - hadoop - } - } + def get: SparkHadoopUtil = instance /** * Given an expiration date (e.g. for Hadoop Delegation Tokens) return a the date diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 73b956ef3e470..cfcdce648d330 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -92,6 +92,12 @@ object SparkSubmit extends CommandLineUtils with Logging { private val CLASS_NOT_FOUND_EXIT_STATUS = 101 + // Following constants are visible for testing. + private[deploy] val YARN_CLUSTER_SUBMIT_CLASS = + "org.apache.spark.deploy.yarn.YarnClusterApplication" + private[deploy] val REST_CLUSTER_SUBMIT_CLASS = classOf[RestSubmissionClientApp].getName() + private[deploy] val STANDALONE_CLUSTER_SUBMIT_CLASS = classOf[ClientApp].getName() + // scalastyle:off println private[spark] def printVersionAndExit(): Unit = { printStream.println("""Welcome to @@ -281,7 +287,7 @@ object SparkSubmit extends CommandLineUtils with Logging { } // Make sure YARN is included in our build if we're trying to use it - if (!Utils.classIsLoadable("org.apache.spark.deploy.yarn.Client") && !Utils.isTesting) { + if (!Utils.classIsLoadable(YARN_CLUSTER_SUBMIT_CLASS) && !Utils.isTesting) { printErrorAndExit( "Could not load YARN classes. " + "This copy of Spark may not have been compiled with YARN support.") @@ -363,10 +369,7 @@ object SparkSubmit extends CommandLineUtils with Logging { args.pyFiles = Option(args.pyFiles).map(resolveGlobPaths(_, hadoopConf)).orNull args.archives = Option(args.archives).map(resolveGlobPaths(_, hadoopConf)).orNull - // This security manager will not need an auth secret, but set a dummy value in case - // spark.authenticate is enabled, otherwise an exception is thrown. - lazy val downloadConf = sparkConf.clone().set(SecurityManager.SPARK_AUTH_SECRET_CONF, "unused") - lazy val secMgr = new SecurityManager(downloadConf) + lazy val secMgr = new SecurityManager(sparkConf) // In client mode, download remote files. var localPrimaryResource: String = null @@ -374,13 +377,13 @@ object SparkSubmit extends CommandLineUtils with Logging { var localPyFiles: String = null if (deployMode == CLIENT) { localPrimaryResource = Option(args.primaryResource).map { - downloadFile(_, targetDir, downloadConf, hadoopConf, secMgr) + downloadFile(_, targetDir, sparkConf, hadoopConf, secMgr) }.orNull localJars = Option(args.jars).map { - downloadFileList(_, targetDir, downloadConf, hadoopConf, secMgr) + downloadFileList(_, targetDir, sparkConf, hadoopConf, secMgr) }.orNull localPyFiles = Option(args.pyFiles).map { - downloadFileList(_, targetDir, downloadConf, hadoopConf, secMgr) + downloadFileList(_, targetDir, sparkConf, hadoopConf, secMgr) }.orNull } @@ -391,8 +394,6 @@ object SparkSubmit extends CommandLineUtils with Logging { // For yarn client mode, since we already download them with above code, so we only need to // figure out the local path and replace the remote one. if (clusterManager == YARN) { - sparkConf.setIfMissing(SecurityManager.SPARK_AUTH_SECRET_CONF, "unused") - val secMgr = new SecurityManager(sparkConf) val forceDownloadSchemes = sparkConf.get(FORCE_DOWNLOAD_SCHEMES) def shouldDownload(scheme: String): Boolean = { @@ -409,7 +410,7 @@ object SparkSubmit extends CommandLineUtils with Logging { if (file.exists()) { file.toURI.toString } else { - downloadFile(resource, targetDir, downloadConf, hadoopConf, secMgr) + downloadFile(resource, targetDir, sparkConf, hadoopConf, secMgr) } case _ => uri.toString } @@ -634,11 +635,11 @@ object SparkSubmit extends CommandLineUtils with Logging { // All Spark parameters are expected to be passed to the client through system properties. if (args.isStandaloneCluster) { if (args.useRest) { - childMainClass = "org.apache.spark.deploy.rest.RestSubmissionClient" + childMainClass = REST_CLUSTER_SUBMIT_CLASS childArgs += (args.primaryResource, args.mainClass) } else { // In legacy standalone cluster mode, use Client as a wrapper around the user class - childMainClass = "org.apache.spark.deploy.Client" + childMainClass = STANDALONE_CLUSTER_SUBMIT_CLASS if (args.supervise) { childArgs += "--supervise" } Option(args.driverMemory).foreach { m => childArgs += ("--memory", m) } Option(args.driverCores).foreach { c => childArgs += ("--cores", c) } @@ -663,7 +664,7 @@ object SparkSubmit extends CommandLineUtils with Logging { // In yarn-cluster mode, use yarn.Client as a wrapper around the user class if (isYarnCluster) { - childMainClass = "org.apache.spark.deploy.yarn.Client" + childMainClass = YARN_CLUSTER_SUBMIT_CLASS if (args.isPython) { childArgs += ("--primary-py-file", args.primaryResource) childArgs += ("--class", "org.apache.spark.deploy.PythonRunner") @@ -684,7 +685,7 @@ object SparkSubmit extends CommandLineUtils with Logging { if (isMesosCluster) { assert(args.useRest, "Mesos cluster mode is only supported through the REST submission API") - childMainClass = "org.apache.spark.deploy.rest.RestSubmissionClient" + childMainClass = REST_CLUSTER_SUBMIT_CLASS if (args.isPython) { // Second argument is main class childArgs += (args.primaryResource, "") diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala index 21cb94142b15b..742a95841a138 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala @@ -32,6 +32,7 @@ import scala.util.control.NonFatal import com.fasterxml.jackson.core.JsonProcessingException import org.apache.spark.{SPARK_VERSION => sparkVersion, SparkConf, SparkException} +import org.apache.spark.deploy.SparkApplication import org.apache.spark.internal.Logging import org.apache.spark.util.Utils @@ -398,9 +399,20 @@ private[spark] object RestSubmissionClient { val PROTOCOL_VERSION = "v1" /** - * Submit an application, assuming Spark parameters are specified through the given config. - * This is abstracted to its own method for testing purposes. + * Filter non-spark environment variables from any environment. */ + private[rest] def filterSystemEnvironment(env: Map[String, String]): Map[String, String] = { + env.filterKeys { k => + // SPARK_HOME is filtered out because it is usually wrong on the remote machine (SPARK-12345) + (k.startsWith("SPARK_") && k != "SPARK_ENV_LOADED" && k != "SPARK_HOME") || + k.startsWith("MESOS_") + } + } +} + +private[spark] class RestSubmissionClientApp extends SparkApplication { + + /** Submits a request to run the application and return the response. Visible for testing. */ def run( appResource: String, mainClass: String, @@ -417,7 +429,7 @@ private[spark] object RestSubmissionClient { client.createSubmission(submitRequest) } - def main(args: Array[String]): Unit = { + override def start(args: Array[String], conf: SparkConf): Unit = { if (args.length < 2) { sys.error("Usage: RestSubmissionClient [app resource] [main class] [app args*]") sys.exit(1) @@ -425,19 +437,8 @@ private[spark] object RestSubmissionClient { val appResource = args(0) val mainClass = args(1) val appArgs = args.slice(2, args.length) - val conf = new SparkConf - val env = filterSystemEnvironment(sys.env) + val env = RestSubmissionClient.filterSystemEnvironment(sys.env) run(appResource, mainClass, appArgs, conf, env) } - /** - * Filter non-spark environment variables from any environment. - */ - private[rest] def filterSystemEnvironment(env: Map[String, String]): Map[String, String] = { - env.filterKeys { k => - // SPARK_HOME is filtered out because it is usually wrong on the remote machine (SPARK-12345) - (k.startsWith("SPARK_") && k != "SPARK_ENV_LOADED" && k != "SPARK_HOME") || - k.startsWith("MESOS_") - } - } } 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 acefc9d2436d0..4c1f92a1bcbf2 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -220,7 +220,9 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { if (driverConf.contains("spark.yarn.credentials.file")) { logInfo("Will periodically update credentials from: " + driverConf.get("spark.yarn.credentials.file")) - SparkHadoopUtil.get.startCredentialUpdater(driverConf) + Utils.classForName("org.apache.spark.deploy.yarn.YarnSparkHadoopUtil") + .getMethod("startCredentialUpdater", classOf[SparkConf]) + .invoke(null, driverConf) } cfg.hadoopDelegationCreds.foreach { tokens => @@ -236,7 +238,11 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { env.rpcEnv.setupEndpoint("WorkerWatcher", new WorkerWatcher(env.rpcEnv, url)) } env.rpcEnv.awaitTermination() - SparkHadoopUtil.get.stopCredentialUpdater() + if (driverConf.contains("spark.yarn.credentials.file")) { + Utils.classForName("org.apache.spark.deploy.yarn.YarnSparkHadoopUtil") + .getMethod("stopCredentialUpdater") + .invoke(null) + } } } 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 930e09d90c2f5..51bf91614c866 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -50,6 +50,7 @@ import org.apache.commons.lang3.SystemUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, FileUtil, Path} import org.apache.hadoop.security.UserGroupInformation +import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.log4j.PropertyConfigurator import org.eclipse.jetty.util.MultiException import org.json4s._ @@ -59,6 +60,7 @@ import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ +import org.apache.spark.launcher.SparkLauncher import org.apache.spark.network.util.JavaUtils import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} @@ -2405,8 +2407,8 @@ private[spark] object Utils extends Logging { */ def getSparkOrYarnConfig(conf: SparkConf, key: String, default: String): String = { val sparkValue = conf.get(key, default) - if (SparkHadoopUtil.get.isYarnMode) { - SparkHadoopUtil.get.newConfiguration(conf).get(key, sparkValue) + if (conf.get(SparkLauncher.SPARK_MASTER, null) == "yarn") { + new YarnConfiguration(SparkHadoopUtil.get.newConfiguration(conf)).get(key, sparkValue) } else { sparkValue } diff --git a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala index 9801b2638cc15..cf59265dd646d 100644 --- a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala @@ -18,7 +18,13 @@ package org.apache.spark import java.io.File +import java.nio.charset.StandardCharsets.UTF_8 +import java.security.PrivilegedExceptionAction +import org.apache.hadoop.security.UserGroupInformation + +import org.apache.spark.internal.config._ +import org.apache.spark.launcher.SparkLauncher import org.apache.spark.security.GroupMappingServiceProvider import org.apache.spark.util.{ResetSystemProperties, SparkConfWithEnv, Utils} @@ -411,8 +417,12 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { test("missing secret authentication key") { val conf = new SparkConf().set("spark.authenticate", "true") + val mgr = new SecurityManager(conf) + intercept[IllegalArgumentException] { + mgr.getSecretKey() + } intercept[IllegalArgumentException] { - new SecurityManager(conf) + mgr.initializeAuth() } } @@ -430,5 +440,24 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { assert(keyFromEnv === new SecurityManager(conf2).getSecretKey()) } + test("secret key generation in yarn mode") { + val conf = new SparkConf() + .set(NETWORK_AUTH_ENABLED, true) + .set(SparkLauncher.SPARK_MASTER, "yarn") + val mgr = new SecurityManager(conf) + + UserGroupInformation.createUserForTesting("authTest", Array()).doAs( + new PrivilegedExceptionAction[Unit]() { + override def run(): Unit = { + mgr.initializeAuth() + val creds = UserGroupInformation.getCurrentUser().getCredentials() + val secret = creds.getSecretKey(SecurityManager.SECRET_LOOKUP_KEY) + assert(secret != null) + assert(new String(secret, UTF_8) === mgr.getSecretKey()) + } + } + ) + } + } diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index d0a34c5cdcf57..e200755e639e1 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -235,7 +235,7 @@ class SparkSubmitSuite childArgsStr should include ("--class org.SomeClass") childArgsStr should include ("--arg arg1 --arg arg2") childArgsStr should include regex ("--jar .*thejar.jar") - mainClass should be ("org.apache.spark.deploy.yarn.Client") + mainClass should be (SparkSubmit.YARN_CLUSTER_SUBMIT_CLASS) // In yarn cluster mode, also adding jars to classpath classpath(0) should endWith ("thejar.jar") @@ -323,11 +323,11 @@ class SparkSubmitSuite val childArgsStr = childArgs.mkString(" ") if (useRest) { childArgsStr should endWith ("thejar.jar org.SomeClass arg1 arg2") - mainClass should be ("org.apache.spark.deploy.rest.RestSubmissionClient") + mainClass should be (SparkSubmit.REST_CLUSTER_SUBMIT_CLASS) } else { childArgsStr should startWith ("--supervise --memory 4g --cores 5") childArgsStr should include regex "launch spark://h:p .*thejar.jar org.SomeClass arg1 arg2" - mainClass should be ("org.apache.spark.deploy.Client") + mainClass should be (SparkSubmit.STANDALONE_CLUSTER_SUBMIT_CLASS) } classpath should have size 0 sys.props("SPARK_SUBMIT") should be ("true") @@ -402,7 +402,7 @@ class SparkSubmitSuite conf.get("spark.executor.memory") should be ("5g") conf.get("spark.master") should be ("yarn") conf.get("spark.submit.deployMode") should be ("cluster") - mainClass should be ("org.apache.spark.deploy.yarn.Client") + mainClass should be (SparkSubmit.YARN_CLUSTER_SUBMIT_CLASS) } test("SPARK-21568 ConsoleProgressBar should be enabled only in shells") { diff --git a/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala index 490baf040491f..e505bc018857d 100644 --- a/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala @@ -92,7 +92,7 @@ class StandaloneRestSubmitSuite extends SparkFunSuite with BeforeAndAfterEach { conf.set("spark.app.name", "dreamer") val appArgs = Array("one", "two", "six") // main method calls this - val response = RestSubmissionClient.run("app-resource", "main-class", appArgs, conf) + val response = new RestSubmissionClientApp().run("app-resource", "main-class", appArgs, conf) val submitResponse = getSubmitResponse(response) assert(submitResponse.action === Utils.getFormattedClassName(submitResponse)) assert(submitResponse.serverSparkVersion === SPARK_VERSION) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 5b8dcd0338cce..9be01f617217b 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -36,6 +36,12 @@ object MimaExcludes { // Exclude rules for 2.3.x lazy val v23excludes = v22excludes ++ Seq( + // SPARK-22372: Make cluster submission use SparkApplication. + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkHadoopUtil.getSecretKeyFromUserCredentials"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkHadoopUtil.isYarnMode"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkHadoopUtil.getCurrentUserCredentials"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkHadoopUtil.addSecretKeyToUserCredentials"), + // SPARK-18085: Better History Server scalability for many / large applications ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.status.api.v1.ExecutorSummary.executorLogs"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.history.HistoryServer.getSparkUI"), diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index ca0aa0ea3bc73..b2576b0d72633 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -56,11 +56,28 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends // TODO: Currently, task to container is computed once (TaskSetManager) - which need not be // optimal as more containers are available. Might need to handle this better. - private val sparkConf = new SparkConf() - private val yarnConf: YarnConfiguration = SparkHadoopUtil.get.newConfiguration(sparkConf) - .asInstanceOf[YarnConfiguration] private val isClusterMode = args.userClass != null + private val sparkConf = new SparkConf() + if (args.propertiesFile != null) { + Utils.getPropertiesFromFile(args.propertiesFile).foreach { case (k, v) => + sparkConf.set(k, v) + } + } + + private val securityMgr = new SecurityManager(sparkConf) + + // Set system properties for each config entry. This covers two use cases: + // - The default configuration stored by the SparkHadoopUtil class + // - The user application creating a new SparkConf in cluster mode + // + // Both cases create a new SparkConf object which reads these configs from system properties. + sparkConf.getAll.foreach { case (k, v) => + sys.props(k) = v + } + + private val yarnConf = new YarnConfiguration(SparkHadoopUtil.newConfiguration(sparkConf)) + private val ugi = { val original = UserGroupInformation.getCurrentUser() @@ -311,7 +328,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends val credentialManager = new YARNHadoopDelegationTokenManager( sparkConf, yarnConf, - conf => YarnSparkHadoopUtil.get.hadoopFSsToAccess(sparkConf, conf)) + conf => YarnSparkHadoopUtil.hadoopFSsToAccess(sparkConf, conf)) val credentialRenewer = new AMCredentialRenewer(sparkConf, yarnConf, credentialManager) @@ -323,13 +340,10 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends credentialRenewerThread.join() } - // Call this to force generation of secret so it gets populated into the Hadoop UGI. - val securityMgr = new SecurityManager(sparkConf) - if (isClusterMode) { - runDriver(securityMgr) + runDriver() } else { - runExecutorLauncher(securityMgr) + runExecutorLauncher() } } catch { case e: Exception => @@ -410,8 +424,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends _sparkConf: SparkConf, _rpcEnv: RpcEnv, driverRef: RpcEndpointRef, - uiAddress: Option[String], - securityMgr: SecurityManager) = { + uiAddress: Option[String]) = { val appId = client.getAttemptId().getApplicationId().toString() val attemptId = client.getAttemptId().getAttemptId().toString() val historyAddress = @@ -463,7 +476,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends YarnSchedulerBackend.ENDPOINT_NAME) } - private def runDriver(securityMgr: SecurityManager): Unit = { + private def runDriver(): Unit = { addAmIpFilter(None) userClassThread = startUserApplication() @@ -479,7 +492,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends val driverRef = createSchedulerRef( sc.getConf.get("spark.driver.host"), sc.getConf.get("spark.driver.port")) - registerAM(sc.getConf, rpcEnv, driverRef, sc.ui.map(_.webUrl), securityMgr) + registerAM(sc.getConf, rpcEnv, driverRef, sc.ui.map(_.webUrl)) registered = true } else { // Sanity check; should never happen in normal operation, since sc should only be null @@ -498,15 +511,14 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends } } - private def runExecutorLauncher(securityMgr: SecurityManager): Unit = { + private def runExecutorLauncher(): Unit = { val hostname = Utils.localHostName val amCores = sparkConf.get(AM_CORES) rpcEnv = RpcEnv.create("sparkYarnAM", hostname, hostname, -1, sparkConf, securityMgr, amCores, true) val driverRef = waitForSparkDriver() addAmIpFilter(Some(driverRef)) - registerAM(sparkConf, rpcEnv, driverRef, sparkConf.getOption("spark.driver.appUIAddress"), - securityMgr) + registerAM(sparkConf, rpcEnv, driverRef, sparkConf.getOption("spark.driver.appUIAddress")) registered = true // In client mode the actor will stop the reporter thread. @@ -686,6 +698,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends if (args.primaryRFile != null && args.primaryRFile.endsWith(".R")) { // TODO(davies): add R dependencies here } + val mainMethod = userClassLoader.loadClass(args.userClass) .getMethod("main", classOf[Array[String]]) @@ -809,15 +822,6 @@ object ApplicationMaster extends Logging { def main(args: Array[String]): Unit = { SignalUtils.registerLogger(log) val amArgs = new ApplicationMasterArguments(args) - - // Load the properties file with the Spark configuration and set entries as system properties, - // so that user code run inside the AM also has access to them. - // Note: we must do this before SparkHadoopUtil instantiated - if (amArgs.propertiesFile != null) { - Utils.getPropertiesFromFile(amArgs.propertiesFile).foreach { case (k, v) => - sys.props(k) = v - } - } master = new ApplicationMaster(amArgs) System.exit(master.run()) } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 99e7d46ca5c96..3781b261a0381 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -48,7 +48,7 @@ import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException import org.apache.hadoop.yarn.util.Records import org.apache.spark.{SecurityManager, SparkConf, SparkException} -import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.{SparkApplication, SparkHadoopUtil} import org.apache.spark.deploy.yarn.config._ import org.apache.spark.deploy.yarn.security.YARNHadoopDelegationTokenManager import org.apache.spark.internal.Logging @@ -58,18 +58,14 @@ import org.apache.spark.util.{CallerContext, Utils} private[spark] class Client( val args: ClientArguments, - val hadoopConf: Configuration, val sparkConf: SparkConf) extends Logging { import Client._ import YarnSparkHadoopUtil._ - def this(clientArgs: ClientArguments, spConf: SparkConf) = - this(clientArgs, SparkHadoopUtil.get.newConfiguration(spConf), spConf) - private val yarnClient = YarnClient.createYarnClient - private val yarnConf = new YarnConfiguration(hadoopConf) + private val hadoopConf = new YarnConfiguration(SparkHadoopUtil.newConfiguration(sparkConf)) private val isClusterMode = sparkConf.get("spark.submit.deployMode", "client") == "cluster" @@ -125,7 +121,7 @@ private[spark] class Client( private val credentialManager = new YARNHadoopDelegationTokenManager( sparkConf, hadoopConf, - conf => YarnSparkHadoopUtil.get.hadoopFSsToAccess(sparkConf, conf)) + conf => YarnSparkHadoopUtil.hadoopFSsToAccess(sparkConf, conf)) def reportLauncherState(state: SparkAppHandle.State): Unit = { launcherBackend.setState(state) @@ -134,8 +130,6 @@ private[spark] class Client( def stop(): Unit = { launcherBackend.close() yarnClient.stop() - // Unset YARN mode system env variable, to allow switching between cluster types. - System.clearProperty("SPARK_YARN_MODE") } /** @@ -152,7 +146,7 @@ private[spark] class Client( // Setup the credentials before doing anything else, // so we have don't have issues at any point. setupCredentials() - yarnClient.init(yarnConf) + yarnClient.init(hadoopConf) yarnClient.start() logInfo("Requesting a new application from cluster with %d NodeManagers" @@ -398,7 +392,7 @@ private[spark] class Client( if (SparkHadoopUtil.get.isProxyUser(currentUser)) { currentUser.addCredentials(credentials) } - logDebug(YarnSparkHadoopUtil.get.dumpTokens(credentials).mkString("\n")) + logDebug(SparkHadoopUtil.get.dumpTokens(credentials).mkString("\n")) } // If we use principal and keytab to login, also credentials can be renewed some time @@ -758,12 +752,14 @@ private[spark] class Client( // Save the YARN configuration into a separate file that will be overlayed on top of the // cluster's Hadoop conf. confStream.putNextEntry(new ZipEntry(SPARK_HADOOP_CONF_FILE)) - yarnConf.writeXml(confStream) + hadoopConf.writeXml(confStream) confStream.closeEntry() - // Save Spark configuration to a file in the archive. + // Save Spark configuration to a file in the archive, but filter out the app's secret. val props = new Properties() - sparkConf.getAll.foreach { case (k, v) => props.setProperty(k, v) } + sparkConf.getAll.foreach { case (k, v) => + props.setProperty(k, v) + } // Override spark.yarn.key to point to the location in distributed cache which will be used // by AM. Option(amKeytabFileName).foreach { k => props.setProperty(KEYTAB.key, k) } @@ -786,8 +782,7 @@ private[spark] class Client( pySparkArchives: Seq[String]): HashMap[String, String] = { logInfo("Setting up the launch environment for our AM container") val env = new HashMap[String, String]() - populateClasspath(args, yarnConf, sparkConf, env, sparkConf.get(DRIVER_CLASS_PATH)) - env("SPARK_YARN_MODE") = "true" + populateClasspath(args, hadoopConf, sparkConf, env, sparkConf.get(DRIVER_CLASS_PATH)) env("SPARK_YARN_STAGING_DIR") = stagingDirPath.toString env("SPARK_USER") = UserGroupInformation.getCurrentUser().getShortUserName() if (loginFromKeytab) { @@ -861,6 +856,7 @@ private[spark] class Client( } else { Nil } + val launchEnv = setupLaunchEnv(appStagingDirPath, pySparkArchives) val localResources = prepareLocalResources(appStagingDirPath, pySparkArchives) @@ -991,7 +987,11 @@ private[spark] class Client( logDebug("YARN AM launch context:") logDebug(s" user class: ${Option(args.userClass).getOrElse("N/A")}") logDebug(" env:") - launchEnv.foreach { case (k, v) => logDebug(s" $k -> $v") } + if (log.isDebugEnabled) { + Utils.redact(sparkConf, launchEnv.toSeq).foreach { case (k, v) => + logDebug(s" $k -> $v") + } + } logDebug(" resources:") localResources.foreach { case (k, v) => logDebug(s" $k -> $v")} logDebug(" command:") @@ -1185,24 +1185,6 @@ private[spark] class Client( private object Client extends Logging { - def main(argStrings: Array[String]) { - if (!sys.props.contains("SPARK_SUBMIT")) { - logWarning("WARNING: This client is deprecated and will be removed in a " + - "future version of Spark. Use ./bin/spark-submit with \"--master yarn\"") - } - - // Set an env variable indicating we are running in YARN mode. - // Note that any env variable with the SPARK_ prefix gets propagated to all (remote) processes - System.setProperty("SPARK_YARN_MODE", "true") - val sparkConf = new SparkConf - // SparkSubmit would use yarn cache to distribute files & jars in yarn mode, - // so remove them from sparkConf here for yarn mode. - sparkConf.remove("spark.jars") - sparkConf.remove("spark.files") - val args = new ClientArguments(argStrings) - new Client(args, sparkConf).run() - } - // Alias for the user jar val APP_JAR_NAME: String = "__app__.jar" @@ -1506,3 +1488,16 @@ private object Client extends Logging { } } + +private[spark] class YarnClusterApplication extends SparkApplication { + + override def start(args: Array[String], conf: SparkConf): Unit = { + // SparkSubmit would use yarn cache to distribute files & jars in yarn mode, + // so remove them from sparkConf here for yarn mode. + conf.remove("spark.jars") + conf.remove("spark.files") + + new Client(new ClientArguments(args), conf).run() + } + +} diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala index 72f4d273ab53b..c1ae12aabb8cc 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala @@ -92,7 +92,7 @@ private[spark] class YarnRMClient extends Logging { /** Returns the attempt ID. */ def getAttemptId(): ApplicationAttemptId = { - YarnSparkHadoopUtil.get.getContainerId.getApplicationAttemptId() + YarnSparkHadoopUtil.getContainerId.getApplicationAttemptId() } /** Returns the configuration for the AmIpFilter to add to the Spark UI. */ diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 9c1472cb50e3a..f406fabd61860 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -17,21 +17,14 @@ package org.apache.spark.deploy.yarn -import java.nio.charset.StandardCharsets.UTF_8 -import java.util.regex.Matcher -import java.util.regex.Pattern +import java.util.regex.{Matcher, Pattern} import scala.collection.mutable.{HashMap, ListBuffer} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.hadoop.io.Text -import org.apache.hadoop.mapred.{JobConf, Master} -import org.apache.hadoop.security.Credentials -import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.yarn.api.ApplicationConstants import org.apache.hadoop.yarn.api.records.{ApplicationAccessType, ContainerId, Priority} -import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.util.ConverterUtils import org.apache.spark.{SecurityManager, SparkConf, SparkException} @@ -43,87 +36,10 @@ import org.apache.spark.internal.config._ import org.apache.spark.launcher.YarnCommandBuilderUtils import org.apache.spark.util.Utils - -/** - * Contains util methods to interact with Hadoop from spark. - */ -class YarnSparkHadoopUtil extends SparkHadoopUtil { +object YarnSparkHadoopUtil { private var credentialUpdater: CredentialUpdater = _ - override def transferCredentials(source: UserGroupInformation, dest: UserGroupInformation) { - dest.addCredentials(source.getCredentials()) - } - - // Note that all params which start with SPARK are propagated all the way through, so if in yarn - // mode, this MUST be set to true. - override def isYarnMode(): Boolean = { true } - - // Return an appropriate (subclass) of Configuration. Creating a config initializes some Hadoop - // subsystems. Always create a new config, don't reuse yarnConf. - override def newConfiguration(conf: SparkConf): Configuration = { - val hadoopConf = new YarnConfiguration(super.newConfiguration(conf)) - hadoopConf.addResource(Client.SPARK_HADOOP_CONF_FILE) - hadoopConf - } - - // Add any user credentials to the job conf which are necessary for running on a secure Hadoop - // cluster - override def addCredentials(conf: JobConf) { - val jobCreds = conf.getCredentials() - jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials()) - } - - override def addSecretKeyToUserCredentials(key: String, secret: String) { - val creds = new Credentials() - creds.addSecretKey(new Text(key), secret.getBytes(UTF_8)) - addCurrentUserCredentials(creds) - } - - override def getSecretKeyFromUserCredentials(key: String): Array[Byte] = { - val credentials = getCurrentUserCredentials() - if (credentials != null) credentials.getSecretKey(new Text(key)) else null - } - - private[spark] override def startCredentialUpdater(sparkConf: SparkConf): Unit = { - val hadoopConf = newConfiguration(sparkConf) - val credentialManager = new YARNHadoopDelegationTokenManager( - sparkConf, - hadoopConf, - conf => YarnSparkHadoopUtil.get.hadoopFSsToAccess(sparkConf, conf)) - credentialUpdater = new CredentialUpdater(sparkConf, hadoopConf, credentialManager) - credentialUpdater.start() - } - - private[spark] override def stopCredentialUpdater(): Unit = { - if (credentialUpdater != null) { - credentialUpdater.stop() - credentialUpdater = null - } - } - - private[spark] def getContainerId: ContainerId = { - val containerIdString = System.getenv(ApplicationConstants.Environment.CONTAINER_ID.name()) - ConverterUtils.toContainerId(containerIdString) - } - - /** The filesystems for which YARN should fetch delegation tokens. */ - private[spark] def hadoopFSsToAccess( - sparkConf: SparkConf, - hadoopConf: Configuration): Set[FileSystem] = { - val filesystemsToAccess = sparkConf.get(FILESYSTEMS_TO_ACCESS) - .map(new Path(_).getFileSystem(hadoopConf)) - .toSet - - val stagingFS = sparkConf.get(STAGING_DIR) - .map(new Path(_).getFileSystem(hadoopConf)) - .getOrElse(FileSystem.get(hadoopConf)) - - filesystemsToAccess + stagingFS - } -} - -object YarnSparkHadoopUtil { // Additional memory overhead // 10% was arrived at experimentally. In the interest of minimizing memory waste while covering // the common cases. Memory overhead tends to grow with container size. @@ -137,14 +53,6 @@ object YarnSparkHadoopUtil { // request types (like map/reduce in hadoop for example) val RM_REQUEST_PRIORITY = Priority.newInstance(1) - def get: YarnSparkHadoopUtil = { - val yarnMode = java.lang.Boolean.parseBoolean( - System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE"))) - if (!yarnMode) { - throw new SparkException("YarnSparkHadoopUtil is not available in non-YARN mode!") - } - SparkHadoopUtil.get.asInstanceOf[YarnSparkHadoopUtil] - } /** * Add a path variable to the given environment map. * If the map already contains this key, append the value to the existing value instead. @@ -277,5 +185,42 @@ object YarnSparkHadoopUtil { securityMgr.getModifyAclsGroups) ) } -} + def getContainerId: ContainerId = { + val containerIdString = System.getenv(ApplicationConstants.Environment.CONTAINER_ID.name()) + ConverterUtils.toContainerId(containerIdString) + } + + /** The filesystems for which YARN should fetch delegation tokens. */ + def hadoopFSsToAccess( + sparkConf: SparkConf, + hadoopConf: Configuration): Set[FileSystem] = { + val filesystemsToAccess = sparkConf.get(FILESYSTEMS_TO_ACCESS) + .map(new Path(_).getFileSystem(hadoopConf)) + .toSet + + val stagingFS = sparkConf.get(STAGING_DIR) + .map(new Path(_).getFileSystem(hadoopConf)) + .getOrElse(FileSystem.get(hadoopConf)) + + filesystemsToAccess + stagingFS + } + + def startCredentialUpdater(sparkConf: SparkConf): Unit = { + val hadoopConf = SparkHadoopUtil.get.newConfiguration(sparkConf) + val credentialManager = new YARNHadoopDelegationTokenManager( + sparkConf, + hadoopConf, + conf => YarnSparkHadoopUtil.hadoopFSsToAccess(sparkConf, conf)) + credentialUpdater = new CredentialUpdater(sparkConf, hadoopConf, credentialManager) + credentialUpdater.start() + } + + def stopCredentialUpdater(): Unit = { + if (credentialUpdater != null) { + credentialUpdater.stop() + credentialUpdater = null + } + } + +} diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala index 6134757a82fdc..eaf2cff111a49 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala @@ -62,7 +62,7 @@ private[yarn] class AMCredentialRenewer( private val credentialRenewerThread: ScheduledExecutorService = ThreadUtils.newDaemonSingleThreadScheduledExecutor("Credential Refresh Thread") - private val hadoopUtil = YarnSparkHadoopUtil.get + private val hadoopUtil = SparkHadoopUtil.get private val credentialsFile = sparkConf.get(CREDENTIALS_FILE_PATH) private val daysToKeepFiles = sparkConf.get(CREDENTIALS_FILE_MAX_RETENTION) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index b722cc401bb73..0c6206eebe41d 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -66,7 +66,7 @@ private[spark] class YarnClientSchedulerBackend( // reads the credentials from HDFS, just like the executors and updates its own credentials // cache. if (conf.contains("spark.yarn.credentials.file")) { - YarnSparkHadoopUtil.get.startCredentialUpdater(conf) + YarnSparkHadoopUtil.startCredentialUpdater(conf) } monitorThread = asyncMonitorApplication() monitorThread.start() @@ -153,7 +153,7 @@ private[spark] class YarnClientSchedulerBackend( client.reportLauncherState(SparkAppHandle.State.FINISHED) super.stop() - YarnSparkHadoopUtil.get.stopCredentialUpdater() + YarnSparkHadoopUtil.stopCredentialUpdater() client.stop() logInfo("Stopped") } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala index e2d477be329c3..62bf9818ee248 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala @@ -41,7 +41,7 @@ private[spark] class YarnClusterSchedulerBackend( var driverLogs: Option[Map[String, String]] = None try { val yarnConf = new YarnConfiguration(sc.hadoopConfiguration) - val containerId = YarnSparkHadoopUtil.get.getContainerId + val containerId = YarnSparkHadoopUtil.getContainerId val httpAddress = System.getenv(Environment.NM_HOST.name()) + ":" + System.getenv(Environment.NM_HTTP_PORT.name()) diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala index 9c3b18e4ec5f3..ac67f2196e0a0 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala @@ -62,18 +62,14 @@ abstract class BaseYarnClusterSuite protected var hadoopConfDir: File = _ private var logConfDir: File = _ - var oldSystemProperties: Properties = null - def newYarnConfig(): YarnConfiguration override def beforeAll() { super.beforeAll() - oldSystemProperties = SerializationUtils.clone(System.getProperties) tempDir = Utils.createTempDir() logConfDir = new File(tempDir, "log4j") logConfDir.mkdir() - System.setProperty("SPARK_YARN_MODE", "true") val logConfFile = new File(logConfDir, "log4j.properties") Files.write(LOG4J_CONF, logConfFile, StandardCharsets.UTF_8) @@ -124,7 +120,6 @@ abstract class BaseYarnClusterSuite try { yarnCluster.stop() } finally { - System.setProperties(oldSystemProperties) super.afterAll() } } diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala index 6cf68427921fd..9d5f5eb621118 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala @@ -24,7 +24,6 @@ import java.util.Properties import scala.collection.JavaConverters._ import scala.collection.mutable.{HashMap => MutableHashMap} -import org.apache.commons.lang3.SerializationUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce.MRJobConfig @@ -36,34 +35,18 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.util.Records import org.mockito.Matchers.{eq => meq, _} import org.mockito.Mockito._ -import org.scalatest.{BeforeAndAfterAll, Matchers} +import org.scalatest.Matchers import org.apache.spark.{SparkConf, SparkFunSuite, TestUtils} import org.apache.spark.deploy.yarn.config._ -import org.apache.spark.util.{ResetSystemProperties, SparkConfWithEnv, Utils} +import org.apache.spark.util.{SparkConfWithEnv, Utils} -class ClientSuite extends SparkFunSuite with Matchers with BeforeAndAfterAll - with ResetSystemProperties { +class ClientSuite extends SparkFunSuite with Matchers { import Client._ var oldSystemProperties: Properties = null - override def beforeAll(): Unit = { - super.beforeAll() - oldSystemProperties = SerializationUtils.clone(System.getProperties) - System.setProperty("SPARK_YARN_MODE", "true") - } - - override def afterAll(): Unit = { - try { - System.setProperties(oldSystemProperties) - oldSystemProperties = null - } finally { - super.afterAll() - } - } - test("default Yarn application classpath") { getDefaultYarnApplicationClasspath should be(Fixtures.knownDefYarnAppCP) } @@ -185,7 +168,6 @@ class ClientSuite extends SparkFunSuite with Matchers with BeforeAndAfterAll } test("configuration and args propagate through createApplicationSubmissionContext") { - val conf = new Configuration() // When parsing tags, duplicates and leading/trailing whitespace should be removed. // Spaces between non-comma strings should be preserved as single tags. Empty strings may or // may not be removed depending on the version of Hadoop being used. @@ -200,7 +182,7 @@ class ClientSuite extends SparkFunSuite with Matchers with BeforeAndAfterAll val getNewApplicationResponse = Records.newRecord(classOf[GetNewApplicationResponse]) val containerLaunchContext = Records.newRecord(classOf[ContainerLaunchContext]) - val client = new Client(args, conf, sparkConf) + val client = new Client(args, sparkConf) client.createApplicationSubmissionContext( new YarnClientApplication(getNewApplicationResponse, appContext), containerLaunchContext) @@ -407,15 +389,14 @@ class ClientSuite extends SparkFunSuite with Matchers with BeforeAndAfterAll private def createClient( sparkConf: SparkConf, - conf: Configuration = new Configuration(), args: Array[String] = Array()): Client = { val clientArgs = new ClientArguments(args) - spy(new Client(clientArgs, conf, sparkConf)) + spy(new Client(clientArgs, sparkConf)) } private def classpath(client: Client): Array[String] = { val env = new MutableHashMap[String, String]() - populateClasspath(null, client.hadoopConf, client.sparkConf, env) + populateClasspath(null, new Configuration(), client.sparkConf, env) classpath(env) } diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index d5de19072ce29..ab0005d7b53a8 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -115,8 +115,13 @@ class YarnClusterSuite extends BaseYarnClusterSuite { )) } - test("run Spark in yarn-cluster mode with using SparkHadoopUtil.conf") { - testYarnAppUseSparkHadoopUtilConf() + test("yarn-cluster should respect conf overrides in SparkHadoopUtil (SPARK-16414)") { + val result = File.createTempFile("result", null, tempDir) + val finalState = runSpark(false, + mainClassName(YarnClusterDriverUseSparkHadoopUtilConf.getClass), + appArgs = Seq("key=value", result.getAbsolutePath()), + extraConf = Map("spark.hadoop.key" -> "value")) + checkResult(finalState, result) } test("run Spark in yarn-client mode with additional jar") { @@ -216,15 +221,6 @@ class YarnClusterSuite extends BaseYarnClusterSuite { checkResult(finalState, result) } - private def testYarnAppUseSparkHadoopUtilConf(): Unit = { - val result = File.createTempFile("result", null, tempDir) - val finalState = runSpark(false, - mainClassName(YarnClusterDriverUseSparkHadoopUtilConf.getClass), - appArgs = Seq("key=value", result.getAbsolutePath()), - extraConf = Map("spark.hadoop.key" -> "value")) - checkResult(finalState, result) - } - private def testWithAddJar(clientMode: Boolean): Unit = { val originalJar = TestUtils.createJarWithFiles(Map("test.resource" -> "ORIGINAL"), tempDir) val driverResult = File.createTempFile("driver", null, tempDir) @@ -424,7 +420,7 @@ private object YarnClusterDriver extends Logging with Matchers { s"Driver logs contain sensitive info (${SECRET_PASSWORD}): \n${log} " ) } - val containerId = YarnSparkHadoopUtil.get.getContainerId + val containerId = YarnSparkHadoopUtil.getContainerId val user = Utils.getCurrentUserName() assert(urlStr.endsWith(s"/node/containerlogs/$containerId/$user/stderr?start=-4096")) } diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala index a057618b39950..f21353aa007c8 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala @@ -71,14 +71,10 @@ class YarnSparkHadoopUtilSuite extends SparkFunSuite with Matchers with Logging test("Yarn configuration override") { val key = "yarn.nodemanager.hostname" - val default = new YarnConfiguration() - val sparkConf = new SparkConf() .set("spark.hadoop." + key, "someHostName") - val yarnConf = new YarnSparkHadoopUtil().newConfiguration(sparkConf) - - yarnConf.getClass() should be (classOf[YarnConfiguration]) - yarnConf.get(key) should not be default.get(key) + val yarnConf = new YarnConfiguration(SparkHadoopUtil.get.newConfiguration(sparkConf)) + yarnConf.get(key) should be ("someHostName") } @@ -145,45 +141,4 @@ class YarnSparkHadoopUtilSuite extends SparkFunSuite with Matchers with Logging } - test("check different hadoop utils based on env variable") { - try { - System.setProperty("SPARK_YARN_MODE", "true") - assert(SparkHadoopUtil.get.getClass === classOf[YarnSparkHadoopUtil]) - System.setProperty("SPARK_YARN_MODE", "false") - assert(SparkHadoopUtil.get.getClass === classOf[SparkHadoopUtil]) - } finally { - System.clearProperty("SPARK_YARN_MODE") - } - } - - - - // This test needs to live here because it depends on isYarnMode returning true, which can only - // happen in the YARN module. - test("security manager token generation") { - try { - System.setProperty("SPARK_YARN_MODE", "true") - val initial = SparkHadoopUtil.get - .getSecretKeyFromUserCredentials(SecurityManager.SECRET_LOOKUP_KEY) - assert(initial === null || initial.length === 0) - - val conf = new SparkConf() - .set(SecurityManager.SPARK_AUTH_CONF, "true") - .set(SecurityManager.SPARK_AUTH_SECRET_CONF, "unused") - val sm = new SecurityManager(conf) - - val generated = SparkHadoopUtil.get - .getSecretKeyFromUserCredentials(SecurityManager.SECRET_LOOKUP_KEY) - assert(generated != null) - val genString = new Text(generated).toString() - assert(genString != "unused") - assert(sm.getSecretKey() === genString) - } finally { - // removeSecretKey() was only added in Hadoop 2.6, so instead we just set the secret - // to an empty string. - SparkHadoopUtil.get.addSecretKeyToUserCredentials(SecurityManager.SECRET_LOOKUP_KEY, "") - System.clearProperty("SPARK_YARN_MODE") - } - } - } diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManagerSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManagerSuite.scala index c918998bde07c..3c7cdc0f1dab8 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManagerSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManagerSuite.scala @@ -31,24 +31,15 @@ class YARNHadoopDelegationTokenManagerSuite extends SparkFunSuite with Matchers override def beforeAll(): Unit = { super.beforeAll() - - System.setProperty("SPARK_YARN_MODE", "true") - sparkConf = new SparkConf() hadoopConf = new Configuration() } - override def afterAll(): Unit = { - super.afterAll() - - System.clearProperty("SPARK_YARN_MODE") - } - test("Correctly loads credential providers") { credentialManager = new YARNHadoopDelegationTokenManager( sparkConf, hadoopConf, - conf => YarnSparkHadoopUtil.get.hadoopFSsToAccess(sparkConf, conf)) + conf => YarnSparkHadoopUtil.hadoopFSsToAccess(sparkConf, conf)) credentialManager.credentialProviders.get("yarn-test") should not be (None) } From dcaac45bc7cdf9c1358e46380fbc18ff871c9f2e Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Thu, 9 Nov 2017 16:28:10 -0800 Subject: [PATCH 20/37] Spark on Kubernetes - basic submission client --- assembly/pom.xml | 10 + .../org/apache/spark/deploy/SparkSubmit.scala | 37 ++- .../spark/deploy/SparkSubmitArguments.scala | 15 ++ .../spark/deploy/SparkSubmitSuite.scala | 26 ++ .../launcher/SparkSubmitOptionParser.java | 6 +- .../org/apache/spark/deploy/k8s/Config.scala | 92 +++++-- .../apache/spark/deploy/k8s/Constants.scala | 28 ++ .../deploy/k8s/KubernetesClientFactory.scala | 104 ++++++++ .../spark/deploy/k8s/submit/Client.scala | 219 ++++++++++++++++ ...DriverConfigurationStepsOrchestrator.scala | 84 ++++++ .../k8s/submit/KubernetesDriverSpec.scala | 47 ++++ .../k8s/submit/LoggingPodStatusWatcher.scala | 184 +++++++++++++ .../deploy/k8s/submit/MainAppResource.scala | 21 ++ .../steps/BaseDriverConfigurationStep.scala | 163 ++++++++++++ .../steps/DriverConfigurationStep.scala | 30 +++ .../DriverKubernetesCredentialsStep.scala | 244 ++++++++++++++++++ .../steps/DriverServiceBootstrapStep.scala | 103 ++++++++ .../k8s/KubernetesClusterManager.scala | 2 +- .../spark/deploy/k8s/submit/ClientSuite.scala | 234 +++++++++++++++++ ...rConfigurationStepsOrchestratorSuite.scala | 61 +++++ .../BaseDriverConfigurationStepSuite.scala | 118 +++++++++ ...DriverKubernetesCredentialsStepSuite.scala | 153 +++++++++++ .../DriverServiceBootstrapStepSuite.scala | 180 +++++++++++++ .../src/main/dockerfiles/driver/Dockerfile | 31 +++ .../src/main/dockerfiles/executor/Dockerfile | 31 +++ .../main/dockerfiles/spark-base/Dockerfile | 43 +++ .../main/dockerfiles/spark-base/entrypoint.sh | 37 +++ 27 files changed, 2279 insertions(+), 24 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesClientFactory.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverSpec.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/MainAppResource.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverConfigurationStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverKubernetesCredentialsStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverServiceBootstrapStep.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStepSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverKubernetesCredentialsStepSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverServiceBootstrapStepSuite.scala create mode 100644 resource-managers/kubernetes/docker/src/main/dockerfiles/driver/Dockerfile create mode 100644 resource-managers/kubernetes/docker/src/main/dockerfiles/executor/Dockerfile create mode 100644 resource-managers/kubernetes/docker/src/main/dockerfiles/spark-base/Dockerfile create mode 100644 resource-managers/kubernetes/docker/src/main/dockerfiles/spark-base/entrypoint.sh diff --git a/assembly/pom.xml b/assembly/pom.xml index 01fe354235e5b..b3b4239771bc3 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -148,6 +148,16 @@ + + kubernetes + + + org.apache.spark + spark-kubernetes_${scala.binary.version} + ${project.version} + + + hive diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index cfcdce648d330..b094751fe721a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -76,7 +76,8 @@ object SparkSubmit extends CommandLineUtils with Logging { private val STANDALONE = 2 private val MESOS = 4 private val LOCAL = 8 - private val ALL_CLUSTER_MGRS = YARN | STANDALONE | MESOS | LOCAL + private val KUBERNETES = 16 + private val ALL_CLUSTER_MGRS = YARN | STANDALONE | MESOS | LOCAL | KUBERNETES // Deploy modes private val CLIENT = 1 @@ -257,6 +258,7 @@ object SparkSubmit extends CommandLineUtils with Logging { YARN case m if m.startsWith("spark") => STANDALONE case m if m.startsWith("mesos") => MESOS + case m if m.startsWith("k8s") => KUBERNETES case m if m.startsWith("local") => LOCAL case _ => printErrorAndExit("Master must either be yarn or start with spark, mesos, local") @@ -302,6 +304,12 @@ object SparkSubmit extends CommandLineUtils with Logging { case (STANDALONE, CLUSTER) if args.isR => printErrorAndExit("Cluster deploy mode is currently not supported for R " + "applications on standalone clusters.") + case (KUBERNETES, CLIENT) => + printErrorAndExit("Client mode is currently not supported for Kubernetes.") + case (KUBERNETES, _) if args.isPython => + printErrorAndExit("Python applications are currently not supported for Kubernetes.") + case (KUBERNETES, _) if args.isR => + printErrorAndExit("R applications are currently not supported for Kubernetes.") case (LOCAL, CLUSTER) => printErrorAndExit("Cluster deploy mode is not compatible with master \"local\"") case (_, CLUSTER) if isShell(args.primaryResource) => @@ -322,6 +330,7 @@ object SparkSubmit extends CommandLineUtils with Logging { val isYarnCluster = clusterManager == YARN && deployMode == CLUSTER val isMesosCluster = clusterManager == MESOS && deployMode == CLUSTER val isStandAloneCluster = clusterManager == STANDALONE && deployMode == CLUSTER + val isKubernetesCluster = clusterManager == KUBERNETES && deployMode == CLUSTER if (!isMesosCluster && !isStandAloneCluster) { // Resolve maven dependencies if there are any and add classpath to jars. Add them to py-files @@ -556,20 +565,24 @@ object SparkSubmit extends CommandLineUtils with Logging { OptionAssigner(args.principal, YARN, ALL_DEPLOY_MODES, confKey = "spark.yarn.principal"), OptionAssigner(args.keytab, YARN, ALL_DEPLOY_MODES, confKey = "spark.yarn.keytab"), + // Kubernetes only + OptionAssigner(args.kubernetesNamespace, KUBERNETES, ALL_DEPLOY_MODES, + confKey = "spark.kubernetes.namespace"), + // Other options - OptionAssigner(args.executorCores, STANDALONE | YARN, ALL_DEPLOY_MODES, + OptionAssigner(args.executorCores, STANDALONE | YARN | KUBERNETES, ALL_DEPLOY_MODES, confKey = "spark.executor.cores"), - OptionAssigner(args.executorMemory, STANDALONE | MESOS | YARN, ALL_DEPLOY_MODES, + OptionAssigner(args.executorMemory, STANDALONE | MESOS | YARN | KUBERNETES, ALL_DEPLOY_MODES, confKey = "spark.executor.memory"), - OptionAssigner(args.totalExecutorCores, STANDALONE | MESOS, ALL_DEPLOY_MODES, + OptionAssigner(args.totalExecutorCores, STANDALONE | MESOS | KUBERNETES, ALL_DEPLOY_MODES, confKey = "spark.cores.max"), OptionAssigner(args.files, LOCAL | STANDALONE | MESOS, ALL_DEPLOY_MODES, confKey = "spark.files"), OptionAssigner(args.jars, LOCAL, CLIENT, confKey = "spark.jars"), OptionAssigner(args.jars, STANDALONE | MESOS, ALL_DEPLOY_MODES, confKey = "spark.jars"), - OptionAssigner(args.driverMemory, STANDALONE | MESOS | YARN, CLUSTER, + OptionAssigner(args.driverMemory, STANDALONE | MESOS | YARN | KUBERNETES, CLUSTER, confKey = "spark.driver.memory"), - OptionAssigner(args.driverCores, STANDALONE | MESOS | YARN, CLUSTER, + OptionAssigner(args.driverCores, STANDALONE | MESOS | YARN | KUBERNETES, CLUSTER, confKey = "spark.driver.cores"), OptionAssigner(args.supervise.toString, STANDALONE | MESOS, CLUSTER, confKey = "spark.driver.supervise"), @@ -703,6 +716,18 @@ object SparkSubmit extends CommandLineUtils with Logging { } } + if (isKubernetesCluster) { + childMainClass = "org.apache.spark.deploy.k8s.submit.Client" + childArgs ++= Array("--primary-java-resource", args.primaryResource) + childArgs ++= Array("--main-class", args.mainClass) + if (args.childArgs != null) { + args.childArgs.foreach { arg => + childArgs += "--arg" + childArgs += arg + } + } + } + // Load any properties specified through --conf and the default properties file for ((k, v) <- args.sparkProperties) { sparkConf.setIfMissing(k, v) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index a7722e4f86023..a00258e94d9d6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -81,6 +81,9 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S var submissionToRequestStatusFor: String = null var useRest: Boolean = true // used internally + // Kubernetes only + var kubernetesNamespace: String = null + /** Default properties present in the currently defined defaults file. */ lazy val defaultSparkProperties: HashMap[String, String] = { val defaultProperties = new HashMap[String, String]() @@ -199,6 +202,10 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S keytab = Option(keytab).orElse(sparkProperties.get("spark.yarn.keytab")).orNull principal = Option(principal).orElse(sparkProperties.get("spark.yarn.principal")).orNull + kubernetesNamespace = Option(kubernetesNamespace) + .orElse(sparkProperties.get("spark.kubernetes.namespace")) + .orNull + // Try to set main class from JAR if no --class argument is given if (mainClass == null && !isPython && !isR && primaryResource != null) { val uri = new URI(primaryResource) @@ -454,6 +461,9 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S case KEYTAB => keytab = value + case KUBERNETES_NAMESPACE => + kubernetesNamespace = value + case HELP => printUsageAndExit(0) @@ -590,6 +600,11 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S | the node running the Application Master via the Secure | Distributed Cache, for renewing the login tickets and the | delegation tokens periodically. + | + | Kubernetes only: + | --kubernetes-namespace NS The namespace in the Kubernetes cluster within which the + | application must be launched. The namespace must already + | exist in the cluster. (Default: default). """.stripMargin ) diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index e200755e639e1..15eff93a10776 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -388,6 +388,32 @@ class SparkSubmitSuite conf.get("spark.ui.enabled") should be ("false") } + test("handles k8s cluster mode") { + val clArgs = Seq( + "--deploy-mode", "cluster", + "--master", "k8s://host:port", + "--executor-memory", "5g", + "--class", "org.SomeClass", + "--kubernetes-namespace", "foo", + "--driver-memory", "4g", + "--conf", "spark.kubernetes.driver.docker.image=bar", + "/home/thejar.jar", + "arg1") + val appArgs = new SparkSubmitArguments(clArgs) + val (childArgs, classpath, conf, mainClass) = prepareSubmitEnvironment(appArgs) + + val childArgsMap = childArgs.grouped(2).map(a => a(0) -> a(1)).toMap + childArgsMap.get("--primary-java-resource") should be (Some("file:/home/thejar.jar")) + childArgsMap.get("--main-class") should be (Some("org.SomeClass")) + childArgsMap.get("--arg") should be (Some("arg1")) + mainClass should be ("org.apache.spark.deploy.k8s.submit.Client") + classpath should have length (0) + conf.get("spark.executor.memory") should be ("5g") + conf.get("spark.driver.memory") should be ("4g") + conf.get("spark.kubernetes.namespace") should be ("foo") + conf.get("spark.kubernetes.driver.docker.image") should be ("bar") + } + test("handles confs with flag equivalents") { val clArgs = Seq( "--deploy-mode", "cluster", diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java index 6767cc5079649..c6717f127eb73 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java @@ -76,13 +76,16 @@ class SparkSubmitOptionParser { protected final String PRINCIPAL = "--principal"; protected final String QUEUE = "--queue"; + // Kubernetes-only options. + protected final String KUBERNETES_NAMESPACE = "--kubernetes-namespace"; + /** * This is the canonical list of spark-submit options. Each entry in the array contains the * different aliases for the same option; the first element of each entry is the "official" * name of the option, passed to {@link #handle(String, String)}. *

* Options not listed here nor in the "switch" list below will result in a call to - * {@link $#handleUnknown(String)}. + * {@link #handleUnknown(String)}. *

* These two arrays are visible for tests. */ @@ -115,6 +118,7 @@ class SparkSubmitOptionParser { { REPOSITORIES }, { STATUS }, { TOTAL_EXECUTOR_CORES }, + { KUBERNETES_NAMESPACE }, }; /** diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index f0742b91987b6..30068e8893070 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -16,6 +16,8 @@ */ package org.apache.spark.deploy.k8s +import java.util.concurrent.TimeUnit + import org.apache.spark.internal.Logging import org.apache.spark.internal.config.ConfigBuilder import org.apache.spark.network.util.ByteUnit @@ -30,6 +32,12 @@ private[spark] object Config extends Logging { .stringConf .createWithDefault("default") + val DRIVER_DOCKER_IMAGE = + ConfigBuilder("spark.kubernetes.driver.docker.image") + .doc("Docker image to use for the driver. Specify this using the standard Docker tag format.") + .stringConf + .createOptional + val EXECUTOR_DOCKER_IMAGE = ConfigBuilder("spark.kubernetes.executor.docker.image") .doc("Docker image to use for the executors. Specify this using the standard Docker tag " + @@ -44,9 +52,9 @@ private[spark] object Config extends Logging { .checkValues(Set("Always", "Never", "IfNotPresent")) .createWithDefault("IfNotPresent") - val APISERVER_AUTH_DRIVER_CONF_PREFIX = + val KUBERNETES_AUTH_DRIVER_CONF_PREFIX = "spark.kubernetes.authenticate.driver" - val APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX = + val KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX = "spark.kubernetes.authenticate.driver.mounted" val OAUTH_TOKEN_CONF_SUFFIX = "oauthToken" val OAUTH_TOKEN_FILE_CONF_SUFFIX = "oauthTokenFile" @@ -55,14 +63,35 @@ private[spark] object Config extends Logging { val CA_CERT_FILE_CONF_SUFFIX = "caCertFile" val KUBERNETES_SERVICE_ACCOUNT_NAME = - ConfigBuilder(s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.serviceAccountName") - .doc("Service account that is used when running the driver pod. The driver pod uses " + - "this service account when requesting executor pods from the API server. If specific " + - "credentials are given for the driver pod to use, the driver will favor " + - "using those credentials instead.") + ConfigBuilder(s"$KUBERNETES_AUTH_DRIVER_CONF_PREFIX.serviceAccountName") + .doc("Service account that is used when running the driver pod. The driver pod uses" + + " this service account when requesting executor pods from the API server. If specific" + + " credentials are given for the driver pod to use, the driver will favor" + + " using those credentials instead.") .stringConf .createOptional + val KUBERNETES_DRIVER_LIMIT_CORES = + ConfigBuilder("spark.kubernetes.driver.limit.cores") + .doc("Specify the hard cpu limit for the driver pod") + .stringConf + .createOptional + + val KUBERNETES_EXECUTOR_LIMIT_CORES = + ConfigBuilder("spark.kubernetes.executor.limit.cores") + .doc("Specify the hard cpu limit for a single executor pod") + .stringConf + .createOptional + + val KUBERNETES_DRIVER_MEMORY_OVERHEAD = + ConfigBuilder("spark.kubernetes.driver.memoryOverhead") + .doc("The amount of off-heap memory (in megabytes) to be allocated for the driver and the" + + " driver submission server. This is memory that accounts for things like VM overheads," + + " interned strings, other native overheads, etc. This tends to grow with the driver's" + + " memory size (typically 6-10%).") + .bytesConf(ByteUnit.MiB) + .createOptional + // Note that while we set a default for this when we start up the // scheduler, the specific default value is dynamically determined // based on the executor memory. @@ -74,9 +103,6 @@ private[spark] object Config extends Logging { .bytesConf(ByteUnit.MiB) .createOptional - val KUBERNETES_EXECUTOR_LABEL_PREFIX = "spark.kubernetes.executor.label." - val KUBERNETES_EXECUTOR_ANNOTATION_PREFIX = "spark.kubernetes.executor.annotation." - val KUBERNETES_DRIVER_POD_NAME = ConfigBuilder("spark.kubernetes.driver.pod.name") .doc("Name of the driver pod.") @@ -104,12 +130,6 @@ private[spark] object Config extends Logging { .checkValue(value => value > 0, "Allocation batch delay should be a positive integer") .createWithDefault(1) - val KUBERNETES_EXECUTOR_LIMIT_CORES = - ConfigBuilder("spark.kubernetes.executor.limit.cores") - .doc("Specify the hard cpu limit for a single executor pod") - .stringConf - .createOptional - val KUBERNETES_EXECUTOR_LOST_REASON_CHECK_MAX_ATTEMPTS = ConfigBuilder("spark.kubernetes.executor.lostCheck.maxAttempts") .doc("Maximum number of attempts allowed for checking the reason of an executor loss " + @@ -119,5 +139,45 @@ private[spark] object Config extends Logging { "must be a positive integer") .createWithDefault(10) + val WAIT_FOR_APP_COMPLETION = + ConfigBuilder("spark.kubernetes.submission.waitAppCompletion") + .doc("In cluster mode, whether to wait for the application to finish before exiting the" + + " launcher process.") + .booleanConf + .createWithDefault(true) + + val REPORT_INTERVAL = + ConfigBuilder("spark.kubernetes.report.interval") + .doc("Interval between reports of the current app status in cluster mode.") + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefaultString("1s") + + val KUBERNETES_AUTH_SUBMISSION_CONF_PREFIX = + "spark.kubernetes.authenticate.submission" + val KUBERNETES_NODE_SELECTOR_PREFIX = "spark.kubernetes.node.selector." + + val KUBERNETES_DRIVER_LABEL_PREFIX = "spark.kubernetes.driver.label." + val KUBERNETES_DRIVER_ANNOTATION_PREFIX = "spark.kubernetes.driver.annotation." + + val KUBERNETES_EXECUTOR_LABEL_PREFIX = "spark.kubernetes.executor.label." + val KUBERNETES_EXECUTOR_ANNOTATION_PREFIX = "spark.kubernetes.executor.annotation." + + val KUBERNETES_DRIVER_ENV_KEY = "spark.kubernetes.driverEnv." + + def getK8sMasterUrl(rawMasterString: String): String = { + if (!rawMasterString.startsWith("k8s://")) { + throw new IllegalArgumentException("Master URL should start with k8s:// in Kubernetes mode.") + } + val masterWithoutK8sPrefix = rawMasterString.replaceFirst("k8s://", "") + if (masterWithoutK8sPrefix.startsWith("http://") + || masterWithoutK8sPrefix.startsWith("https://")) { + masterWithoutK8sPrefix + } else { + val resolvedURL = s"https://$masterWithoutK8sPrefix" + logInfo("No scheme specified for kubernetes master URL, so defaulting to https. Resolved" + + s" URL is $resolvedURL") + resolvedURL + } + } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala index 4ddeefb15a89d..0b91145405d3a 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala @@ -25,9 +25,30 @@ private[spark] object Constants { val SPARK_POD_DRIVER_ROLE = "driver" val SPARK_POD_EXECUTOR_ROLE = "executor" + // Annotations + val SPARK_APP_NAME_ANNOTATION = "spark-app-name" + + // Credentials secrets + val DRIVER_CREDENTIALS_SECRETS_BASE_DIR = + "/mnt/secrets/spark-kubernetes-credentials" + val DRIVER_CREDENTIALS_CA_CERT_SECRET_NAME = "ca-cert" + val DRIVER_CREDENTIALS_CA_CERT_PATH = + s"$DRIVER_CREDENTIALS_SECRETS_BASE_DIR/$DRIVER_CREDENTIALS_CA_CERT_SECRET_NAME" + val DRIVER_CREDENTIALS_CLIENT_KEY_SECRET_NAME = "client-key" + val DRIVER_CREDENTIALS_CLIENT_KEY_PATH = + s"$DRIVER_CREDENTIALS_SECRETS_BASE_DIR/$DRIVER_CREDENTIALS_CLIENT_KEY_SECRET_NAME" + val DRIVER_CREDENTIALS_CLIENT_CERT_SECRET_NAME = "client-cert" + val DRIVER_CREDENTIALS_CLIENT_CERT_PATH = + s"$DRIVER_CREDENTIALS_SECRETS_BASE_DIR/$DRIVER_CREDENTIALS_CLIENT_CERT_SECRET_NAME" + val DRIVER_CREDENTIALS_OAUTH_TOKEN_SECRET_NAME = "oauth-token" + val DRIVER_CREDENTIALS_OAUTH_TOKEN_PATH = + s"$DRIVER_CREDENTIALS_SECRETS_BASE_DIR/$DRIVER_CREDENTIALS_OAUTH_TOKEN_SECRET_NAME" + val DRIVER_CREDENTIALS_SECRET_VOLUME_NAME = "kubernetes-credentials" + // Default and fixed ports val DEFAULT_DRIVER_PORT = 7078 val DEFAULT_BLOCKMANAGER_PORT = 7079 + val DRIVER_PORT_NAME = "driver-rpc-port" val BLOCK_MANAGER_PORT_NAME = "blockmanager" val EXECUTOR_PORT_NAME = "executor" @@ -42,9 +63,16 @@ private[spark] object Constants { val ENV_EXECUTOR_EXTRA_CLASSPATH = "SPARK_EXECUTOR_EXTRA_CLASSPATH" val ENV_MOUNTED_CLASSPATH = "SPARK_MOUNTED_CLASSPATH" val ENV_JAVA_OPT_PREFIX = "SPARK_JAVA_OPT_" + val ENV_SUBMIT_EXTRA_CLASSPATH = "SPARK_SUBMIT_EXTRA_CLASSPATH" + val ENV_DRIVER_MAIN_CLASS = "SPARK_DRIVER_CLASS" + val ENV_DRIVER_ARGS = "SPARK_DRIVER_ARGS" + val ENV_DRIVER_JAVA_OPTS = "SPARK_DRIVER_JAVA_OPTS" + val ENV_DRIVER_BIND_ADDRESS = "SPARK_DRIVER_BIND_ADDRESS" + val ENV_DRIVER_MEMORY = "SPARK_DRIVER_MEMORY" // Miscellaneous val KUBERNETES_MASTER_INTERNAL_URL = "https://kubernetes.default.svc" + val DRIVER_CONTAINER_NAME = "spark-kubernetes-driver" val MEMORY_OVERHEAD_FACTOR = 0.10 val MEMORY_OVERHEAD_MIN_MIB = 384L } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesClientFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesClientFactory.scala new file mode 100644 index 0000000000000..cc3b61efe08d6 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesClientFactory.scala @@ -0,0 +1,104 @@ +/* + * 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.deploy.k8s + +import java.io.File + +import com.google.common.base.Charsets +import com.google.common.io.Files +import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient, KubernetesClient} +import io.fabric8.kubernetes.client.utils.HttpClientUtils +import okhttp3.Dispatcher + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.util.ThreadUtils + +/** + * Spark-opinionated builder for Kubernetes clients. It uses a prefix plus common suffixes to + * parse configuration keys, similar to the manner in which Spark's SecurityManager parses SSL + * options for different components. + */ +private[spark] object KubernetesClientFactory { + + def createKubernetesClient( + master: String, + namespace: Option[String], + kubernetesAuthConfPrefix: String, + sparkConf: SparkConf, + maybeServiceAccountToken: Option[File], + maybeServiceAccountCaCert: Option[File]): KubernetesClient = { + val oauthTokenFileConf = s"$kubernetesAuthConfPrefix.$OAUTH_TOKEN_FILE_CONF_SUFFIX" + val oauthTokenConf = s"$kubernetesAuthConfPrefix.$OAUTH_TOKEN_CONF_SUFFIX" + val oauthTokenFile = sparkConf.getOption(oauthTokenFileConf) + .map(new File(_)) + .orElse(maybeServiceAccountToken) + val oauthTokenValue = sparkConf.getOption(oauthTokenConf) + ConfigurationUtils.requireNandDefined( + oauthTokenFile, + oauthTokenValue, + s"Cannot specify OAuth token through both a file $oauthTokenFileConf and a" + + s" value $oauthTokenConf.") + + val caCertFile = sparkConf + .getOption(s"$kubernetesAuthConfPrefix.$CA_CERT_FILE_CONF_SUFFIX") + .orElse(maybeServiceAccountCaCert.map(_.getAbsolutePath)) + val clientKeyFile = sparkConf + .getOption(s"$kubernetesAuthConfPrefix.$CLIENT_KEY_FILE_CONF_SUFFIX") + val clientCertFile = sparkConf + .getOption(s"$kubernetesAuthConfPrefix.$CLIENT_CERT_FILE_CONF_SUFFIX") + + val config = new ConfigBuilder() + .withApiVersion("v1") + .withMasterUrl(master) + .withWebsocketPingInterval(0) + .withOption(oauthTokenValue) { + (token, configBuilder) => configBuilder.withOauthToken(token) + }.withOption(oauthTokenFile) { + (file, configBuilder) => + configBuilder.withOauthToken(Files.toString(file, Charsets.UTF_8)) + }.withOption(caCertFile) { + (file, configBuilder) => configBuilder.withCaCertFile(file) + }.withOption(clientKeyFile) { + (file, configBuilder) => configBuilder.withClientKeyFile(file) + }.withOption(clientCertFile) { + (file, configBuilder) => configBuilder.withClientCertFile(file) + }.withOption(namespace) { + (ns, configBuilder) => configBuilder.withNamespace(ns) + }.build() + + val baseHttpClient = HttpClientUtils.createHttpClient(config) + val dispatcher = new Dispatcher( + ThreadUtils.newDaemonCachedThreadPool("kubernetes-dispatcher")) + val httpClientWithCustomDispatcher = baseHttpClient.newBuilder() + .dispatcher(dispatcher) + .build() + new DefaultKubernetesClient(httpClientWithCustomDispatcher, config) + } + + private implicit class OptionConfigurableConfigBuilder(val configBuilder: ConfigBuilder) + extends AnyVal { + + def withOption[T] + (option: Option[T]) + (configurator: ((T, ConfigBuilder) => ConfigBuilder)): ConfigBuilder = { + option.map { opt => + configurator(opt, configBuilder) + }.getOrElse(configBuilder) + } + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala new file mode 100644 index 0000000000000..fe2be3dc6cd68 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala @@ -0,0 +1,219 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.submit + +import java.util.{Collections, UUID} + +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.util.control.NonFatal + +import io.fabric8.kubernetes.api.model._ +import io.fabric8.kubernetes.client.KubernetesClient + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.KubernetesClientFactory +import org.apache.spark.deploy.k8s.submit.steps.DriverConfigurationStep +import org.apache.spark.internal.Logging +import org.apache.spark.util.Utils + +private[spark] case class ClientArguments( + mainAppResource: MainAppResource, + mainClass: String, + driverArgs: Array[String]) + +private[spark] object ClientArguments { + + def fromCommandLineArgs(args: Array[String]): ClientArguments = { + var mainAppResource: Option[MainAppResource] = None + var mainClass: Option[String] = None + val driverArgs = mutable.Buffer.empty[String] + + args.sliding(2, 2).toList.collect { + case Array("--primary-java-resource", primaryJavaResource: String) => + mainAppResource = Some(JavaMainAppResource(primaryJavaResource)) + case Array("--main-class", clazz: String) => + mainClass = Some(clazz) + case Array("--arg", arg: String) => + driverArgs += arg + case other => + val invalid = other.mkString(" ") + throw new RuntimeException(s"Unknown arguments: $invalid") + } + + require(mainAppResource.isDefined, + "Main app resource must be defined by either --primary-py-file or --primary-java-resource.") + require(mainClass.isDefined, "Main class must be specified via --main-class") + + ClientArguments( + mainAppResource.get, + mainClass.get, + driverArgs.toArray) + } +} + +private[spark] class Client( + submissionSteps: Seq[DriverConfigurationStep], + submissionSparkConf: SparkConf, + kubernetesClient: KubernetesClient, + waitForAppCompletion: Boolean, + appName: String, + loggingPodStatusWatcher: LoggingPodStatusWatcher) extends Logging { + + private val driverJavaOptions = submissionSparkConf.get( + org.apache.spark.internal.config.DRIVER_JAVA_OPTIONS) + + /** + * Run command that initializes a DriverSpec that will be updated after each + * DriverConfigurationStep in the sequence that is passed in. The final KubernetesDriverSpec + * will be used to build the Driver Container, Driver Pod, and Kubernetes Resources + */ + def run(): Unit = { + var currentDriverSpec = KubernetesDriverSpec.initialSpec(submissionSparkConf) + // submissionSteps contain steps necessary to take, to resolve varying + // client arguments that are passed in, created by orchestrator + for (nextStep <- submissionSteps) { + currentDriverSpec = nextStep.configureDriver(currentDriverSpec) + } + + val resolvedDriverJavaOpts = currentDriverSpec + .driverSparkConf + // Remove this as the options are instead extracted and set individually below using + // environment variables with prefix SPARK_JAVA_OPT_. + .remove(org.apache.spark.internal.config.DRIVER_JAVA_OPTIONS) + .getAll + .map { + case (confKey, confValue) => s"-D$confKey=$confValue" + } ++ driverJavaOptions.map(Utils.splitCommandString).getOrElse(Seq.empty) + val driverJavaOptsEnvs: Seq[EnvVar] = resolvedDriverJavaOpts.zipWithIndex.map { + case (option, index) => new EnvVarBuilder() + .withName(s"$ENV_JAVA_OPT_PREFIX$index") + .withValue(option) + .build() + } + + val resolvedDriverContainer = new ContainerBuilder(currentDriverSpec.driverContainer) + .addAllToEnv(driverJavaOptsEnvs.asJava) + .build() + val resolvedDriverPod = new PodBuilder(currentDriverSpec.driverPod) + .editSpec() + .addToContainers(resolvedDriverContainer) + .endSpec() + .build() + + Utils.tryWithResource( + kubernetesClient + .pods() + .withName(resolvedDriverPod.getMetadata.getName) + .watch(loggingPodStatusWatcher)) { _ => + val createdDriverPod = kubernetesClient.pods().create(resolvedDriverPod) + try { + if (currentDriverSpec.otherKubernetesResources.nonEmpty) { + val otherKubernetesResources = currentDriverSpec.otherKubernetesResources + addDriverOwnerReference(createdDriverPod, otherKubernetesResources) + kubernetesClient.resourceList(otherKubernetesResources: _*).createOrReplace() + } + } catch { + case NonFatal(e) => + kubernetesClient.pods().delete(createdDriverPod) + throw e + } + + if (waitForAppCompletion) { + logInfo(s"Waiting for application $appName to finish...") + loggingPodStatusWatcher.awaitCompletion() + logInfo(s"Application $appName finished.") + } else { + logInfo(s"Deployed Spark application $appName into Kubernetes.") + } + } + } + + // Add a OwnerReference to the given resources making the driver pod an owner of them so when + // the driver pod is deleted, the resources are garbage collected. + private def addDriverOwnerReference(driverPod: Pod, resources: Seq[HasMetadata]): Unit = { + val driverPodOwnerReference = new OwnerReferenceBuilder() + .withName(driverPod.getMetadata.getName) + .withApiVersion(driverPod.getApiVersion) + .withUid(driverPod.getMetadata.getUid) + .withKind(driverPod.getKind) + .withController(true) + .build() + resources.foreach { resource => + val originalMetadata = resource.getMetadata + originalMetadata.setOwnerReferences(Collections.singletonList(driverPodOwnerReference)) + } + } +} + +private[spark] object Client { + + def run(sparkConf: SparkConf, clientArguments: ClientArguments): Unit = { + val namespace = sparkConf.get(KUBERNETES_NAMESPACE) + val kubernetesAppId = s"spark-${UUID.randomUUID().toString.replaceAll("-", "")}" + val launchTime = System.currentTimeMillis() + val waitForAppCompletion = sparkConf.get(WAIT_FOR_APP_COMPLETION) + val appName = sparkConf.getOption("spark.app.name").getOrElse("spark") + val master = getK8sMasterUrl(sparkConf.get("spark.master")) + val loggingInterval = Option(sparkConf.get(REPORT_INTERVAL)).filter(_ => waitForAppCompletion) + + val loggingPodStatusWatcher = new LoggingPodStatusWatcherImpl( + kubernetesAppId, loggingInterval) + + val configurationStepsOrchestrator = new DriverConfigurationStepsOrchestrator( + namespace, + kubernetesAppId, + launchTime, + clientArguments.mainAppResource, + appName, + clientArguments.mainClass, + clientArguments.driverArgs, + sparkConf) + + Utils.tryWithResource(KubernetesClientFactory.createKubernetesClient( + master, + Some(namespace), + KUBERNETES_AUTH_SUBMISSION_CONF_PREFIX, + sparkConf, + None, + None)) { + kubernetesClient => + val client = new Client( + configurationStepsOrchestrator.getAllConfigurationSteps(), + sparkConf, + kubernetesClient, + waitForAppCompletion, + appName, + loggingPodStatusWatcher) + client.run() + } + } + + /** + * Entry point from SparkSubmit in spark-core + * + * @param args Array of strings that have interchanging values that will be + * parsed by ClientArguments with the identifiers that precede the values + */ + def main(args: Array[String]): Unit = { + val parsedArguments = ClientArguments.fromCommandLineArgs(args) + val sparkConf = new SparkConf() + run(sparkConf, parsedArguments) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala new file mode 100644 index 0000000000000..36d428f3574bf --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.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.deploy.k8s.submit + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.ConfigurationUtils +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.submit.steps.{BaseDriverConfigurationStep, DriverConfigurationStep, DriverKubernetesCredentialsStep, DriverServiceBootstrapStep} +import org.apache.spark.util.SystemClock + +/** + * Constructs the complete list of driver configuration steps to run to deploy the Spark driver. + */ +private[spark] class DriverConfigurationStepsOrchestrator( + namespace: String, + kubernetesAppId: String, + launchTime: Long, + mainAppResource: MainAppResource, + appName: String, + mainClass: String, + appArgs: Array[String], + submissionSparkConf: SparkConf) { + + // The resource name prefix is derived from the application name, making it easy to connect the + // names of the Kubernetes resources from e.g. kubectl or the Kubernetes dashboard to the + // application the user submitted. However, we can't use the application name in the label, as + // label values are considerably restrictive, e.g. must be no longer than 63 characters in + // length. So we generate a separate identifier for the app ID itself, and bookkeeping that + // requires finding "all pods for this application" should use the kubernetesAppId. + private val kubernetesResourceNamePrefix = + s"$appName-$launchTime".toLowerCase.replaceAll("\\.", "-") + private val dockerImagePullPolicy = submissionSparkConf.get(DOCKER_IMAGE_PULL_POLICY) + + def getAllConfigurationSteps(): Seq[DriverConfigurationStep] = { + val driverCustomLabels = ConfigurationUtils.parsePrefixedKeyValuePairs( + submissionSparkConf, + KUBERNETES_DRIVER_LABEL_PREFIX) + require(!driverCustomLabels.contains(SPARK_APP_ID_LABEL), "Label with key " + + s" $SPARK_APP_ID_LABEL is not allowed as it is reserved for Spark bookkeeping" + + " operations.") + val allDriverLabels = driverCustomLabels ++ Map( + SPARK_APP_ID_LABEL -> kubernetesAppId, + SPARK_ROLE_LABEL -> SPARK_POD_DRIVER_ROLE) + + val initialSubmissionStep = new BaseDriverConfigurationStep( + kubernetesAppId, + kubernetesResourceNamePrefix, + allDriverLabels, + dockerImagePullPolicy, + appName, + mainClass, + appArgs, + submissionSparkConf) + + val driverAddressStep = new DriverServiceBootstrapStep( + kubernetesResourceNamePrefix, + allDriverLabels, + submissionSparkConf, + new SystemClock) + + val kubernetesCredentialsStep = new DriverKubernetesCredentialsStep( + submissionSparkConf, kubernetesResourceNamePrefix) + + Seq( + initialSubmissionStep, + driverAddressStep, + kubernetesCredentialsStep) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverSpec.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverSpec.scala new file mode 100644 index 0000000000000..db13f09387ef9 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverSpec.scala @@ -0,0 +1,47 @@ +/* + * 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.deploy.k8s.submit + +import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, HasMetadata, Pod, PodBuilder} + +import org.apache.spark.SparkConf + +/** + * Represents the components and characteristics of a Spark driver. The driver can be considered + * as being comprised of the driver pod itself, any other Kubernetes resources that the driver + * pod depends on, and the SparkConf that should be supplied to the Spark application. The driver + * container should be operated on via the specific field of this case class as opposed to trying + * to edit the container directly on the pod. The driver container should be attached at the + * end of executing all submission steps. + */ +private[spark] case class KubernetesDriverSpec( + driverPod: Pod, + driverContainer: Container, + otherKubernetesResources: Seq[HasMetadata], + driverSparkConf: SparkConf) + +private[spark] object KubernetesDriverSpec { + def initialSpec(initialSparkConf: SparkConf): KubernetesDriverSpec = { + KubernetesDriverSpec( + // Set new metadata and a new spec so that submission steps can use + // PodBuilder#editMetadata() and/or PodBuilder#editSpec() safely. + new PodBuilder().withNewMetadata().endMetadata().withNewSpec().endSpec().build(), + new ContainerBuilder().build(), + Seq.empty[HasMetadata], + initialSparkConf.clone()) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala new file mode 100644 index 0000000000000..be8a32af00afe --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala @@ -0,0 +1,184 @@ +/* + * 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.deploy.k8s.submit + +import java.util.concurrent.{CountDownLatch, TimeUnit} + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model.{ContainerStateRunning, ContainerStateTerminated, ContainerStateWaiting, ContainerStatus, Pod, Time} +import io.fabric8.kubernetes.client.{KubernetesClientException, Watcher} +import io.fabric8.kubernetes.client.Watcher.Action + +import org.apache.spark.SparkException +import org.apache.spark.internal.Logging +import org.apache.spark.util.ThreadUtils + +private[k8s] trait LoggingPodStatusWatcher extends Watcher[Pod] { + def awaitCompletion(): Unit +} + +/** + * A monitor for the running Kubernetes pod of a Spark application. Status logging occurs on + * every state change and also at an interval for liveness. + * + * @param appId application ID. + * @param maybeLoggingInterval ms between each state request. If provided, must be a positive + * number. + */ +private[k8s] class LoggingPodStatusWatcherImpl( + appId: String, maybeLoggingInterval: Option[Long]) + extends LoggingPodStatusWatcher with Logging { + + private val podCompletedFuture = new CountDownLatch(1) + // start timer for periodic logging + private val scheduler = + ThreadUtils.newDaemonSingleThreadScheduledExecutor("logging-pod-status-watcher") + private val logRunnable: Runnable = new Runnable { + override def run() = logShortStatus() + } + + private var pod = Option.empty[Pod] + + private def phase: String = pod.map(_.getStatus.getPhase).getOrElse("unknown") + + def start(): Unit = { + maybeLoggingInterval.foreach { interval => + require(interval > 0, s"Logging interval must be a positive time value, got: $interval ms.") + scheduler.scheduleAtFixedRate(logRunnable, 0, interval, TimeUnit.MILLISECONDS) + } + } + + override def eventReceived(action: Action, pod: Pod): Unit = { + this.pod = Option(pod) + action match { + case Action.DELETED => + closeWatch() + + case Action.ERROR => + closeWatch() + + case _ => + logLongStatus() + if (hasCompleted()) { + closeWatch() + } + } + } + + override def onClose(e: KubernetesClientException): Unit = { + logDebug(s"Stopping watching application $appId with last-observed phase $phase") + closeWatch() + } + + private def logShortStatus() = { + logInfo(s"Application status for $appId (phase: $phase)") + } + + private def logLongStatus() = { + logInfo("State changed, new state: " + pod.map(formatPodState).getOrElse("unknown")) + } + + private def hasCompleted(): Boolean = { + phase == "Succeeded" || phase == "Failed" + } + + private def closeWatch(): Unit = { + podCompletedFuture.countDown() + scheduler.shutdown() + } + + private def formatPodState(pod: Pod): String = { + // TODO include specific container state + val details = Seq[(String, String)]( + // pod metadata + ("pod name", pod.getMetadata.getName), + ("namespace", pod.getMetadata.getNamespace), + ("labels", pod.getMetadata.getLabels.asScala.mkString(", ")), + ("pod uid", pod.getMetadata.getUid), + ("creation time", formatTime(pod.getMetadata.getCreationTimestamp)), + + // spec details + ("service account name", pod.getSpec.getServiceAccountName), + ("volumes", pod.getSpec.getVolumes.asScala.map(_.getName).mkString(", ")), + ("node name", pod.getSpec.getNodeName), + + // status + ("start time", formatTime(pod.getStatus.getStartTime)), + ("container images", + pod.getStatus.getContainerStatuses + .asScala + .map(_.getImage) + .mkString(", ")), + ("phase", pod.getStatus.getPhase), + ("status", pod.getStatus.getContainerStatuses.toString) + ) + + formatPairsBundle(details) + } + + private def formatPairsBundle(pairs: Seq[(String, String)]) = { + // Use more loggable format if value is null or empty + pairs.map { + case (k, v) => s"\n\t $k: ${Option(v).filter(_.nonEmpty).getOrElse("N/A")}" + }.mkString("") + } + + override def awaitCompletion(): Unit = { + podCompletedFuture.await() + logInfo(pod.map { p => + s"Container final statuses:\n\n${containersDescription(p)}" + }.getOrElse("No containers were found in the driver pod.")) + } + + private def containersDescription(p: Pod): String = { + p.getStatus.getContainerStatuses.asScala.map { status => + Seq( + ("Container name", status.getName), + ("Container image", status.getImage)) ++ + containerStatusDescription(status) + }.map(formatPairsBundle).mkString("\n\n") + } + + private def containerStatusDescription( + containerStatus: ContainerStatus): Seq[(String, String)] = { + val state = containerStatus.getState + Option(state.getRunning) + .orElse(Option(state.getTerminated)) + .orElse(Option(state.getWaiting)) + .map { + case running: ContainerStateRunning => + Seq( + ("Container state", "Running"), + ("Container started at", formatTime(running.getStartedAt))) + case waiting: ContainerStateWaiting => + Seq( + ("Container state", "Waiting"), + ("Pending reason", waiting.getReason)) + case terminated: ContainerStateTerminated => + Seq( + ("Container state", "Terminated"), + ("Exit code", terminated.getExitCode.toString)) + case unknown => + throw new SparkException(s"Unexpected container status type ${unknown.getClass}.") + }.getOrElse(Seq(("Container state", "N/A"))) + } + + private def formatTime(time: Time): String = { + if (time != null) time.getTime else "N/A" + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/MainAppResource.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/MainAppResource.scala new file mode 100644 index 0000000000000..cca9f4627a1f6 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/MainAppResource.scala @@ -0,0 +1,21 @@ +/* + * 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.deploy.k8s.submit + +private[spark] sealed trait MainAppResource + +private[spark] case class JavaMainAppResource(primaryResource: String) extends MainAppResource diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStep.scala new file mode 100644 index 0000000000000..398ddbeffc4dc --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStep.scala @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.submit.steps + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model.{ContainerBuilder, EnvVarBuilder, EnvVarSourceBuilder, PodBuilder, QuantityBuilder} + +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.ConfigurationUtils +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.submit.KubernetesDriverSpec + +/** + * Represents the initial setup required for the driver. + */ +private[spark] class BaseDriverConfigurationStep( + kubernetesAppId: String, + kubernetesResourceNamePrefix: String, + driverLabels: Map[String, String], + dockerImagePullPolicy: String, + appName: String, + mainClass: String, + appArgs: Array[String], + submissionSparkConf: SparkConf) extends DriverConfigurationStep { + + private val kubernetesDriverPodName = submissionSparkConf.get(KUBERNETES_DRIVER_POD_NAME) + .getOrElse(s"$kubernetesResourceNamePrefix-driver") + + private val driverExtraClasspath = submissionSparkConf.get( + org.apache.spark.internal.config.DRIVER_CLASS_PATH) + + private val driverDockerImage = submissionSparkConf + .get(DRIVER_DOCKER_IMAGE) + .getOrElse(throw new SparkException("Must specify the driver Docker image")) + + // CPU settings + private val driverCpuCores = submissionSparkConf.getOption("spark.driver.cores").getOrElse("1") + private val driverLimitCores = submissionSparkConf.get(KUBERNETES_DRIVER_LIMIT_CORES) + + // Memory settings + private val driverMemoryMiB = submissionSparkConf.get( + org.apache.spark.internal.config.DRIVER_MEMORY) + private val driverMemoryString = submissionSparkConf.get( + org.apache.spark.internal.config.DRIVER_MEMORY.key, + org.apache.spark.internal.config.DRIVER_MEMORY.defaultValueString) + private val memoryOverheadMiB = submissionSparkConf + .get(KUBERNETES_DRIVER_MEMORY_OVERHEAD) + .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * driverMemoryMiB).toInt, + MEMORY_OVERHEAD_MIN_MIB)) + private val driverContainerMemoryWithOverheadMiB = driverMemoryMiB + memoryOverheadMiB + + override def configureDriver( + driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { + val driverExtraClasspathEnv = driverExtraClasspath.map { classPath => + new EnvVarBuilder() + .withName(ENV_SUBMIT_EXTRA_CLASSPATH) + .withValue(classPath) + .build() + } + + val driverCustomAnnotations = ConfigurationUtils + .parsePrefixedKeyValuePairs( + submissionSparkConf, + KUBERNETES_DRIVER_ANNOTATION_PREFIX) + require(!driverCustomAnnotations.contains(SPARK_APP_NAME_ANNOTATION), + s"Annotation with key $SPARK_APP_NAME_ANNOTATION is not allowed as it is reserved for" + + " Spark bookkeeping operations.") + + val driverCustomEnvs = submissionSparkConf.getAllWithPrefix(KUBERNETES_DRIVER_ENV_KEY).toSeq + .map(env => new EnvVarBuilder() + .withName(env._1) + .withValue(env._2) + .build()) + + val allDriverAnnotations = driverCustomAnnotations ++ Map(SPARK_APP_NAME_ANNOTATION -> appName) + + val nodeSelector = ConfigurationUtils.parsePrefixedKeyValuePairs( + submissionSparkConf, KUBERNETES_NODE_SELECTOR_PREFIX) + + val driverCpuQuantity = new QuantityBuilder(false) + .withAmount(driverCpuCores) + .build() + val driverMemoryQuantity = new QuantityBuilder(false) + .withAmount(s"${driverMemoryMiB}Mi") + .build() + val driverMemoryLimitQuantity = new QuantityBuilder(false) + .withAmount(s"${driverContainerMemoryWithOverheadMiB}Mi") + .build() + val maybeCpuLimitQuantity = driverLimitCores.map { limitCores => + ("cpu", new QuantityBuilder(false).withAmount(limitCores).build()) + } + + val driverContainer = new ContainerBuilder(driverSpec.driverContainer) + .withName(DRIVER_CONTAINER_NAME) + .withImage(driverDockerImage) + .withImagePullPolicy(dockerImagePullPolicy) + .addAllToEnv(driverCustomEnvs.asJava) + .addToEnv(driverExtraClasspathEnv.toSeq: _*) + .addNewEnv() + .withName(ENV_DRIVER_MEMORY) + .withValue(driverMemoryString) + .endEnv() + .addNewEnv() + .withName(ENV_DRIVER_MAIN_CLASS) + .withValue(mainClass) + .endEnv() + .addNewEnv() + .withName(ENV_DRIVER_ARGS) + .withValue(appArgs.mkString(" ")) + .endEnv() + .addNewEnv() + .withName(ENV_DRIVER_BIND_ADDRESS) + .withValueFrom(new EnvVarSourceBuilder() + .withNewFieldRef("v1", "status.podIP") + .build()) + .endEnv() + .withNewResources() + .addToRequests("cpu", driverCpuQuantity) + .addToRequests("memory", driverMemoryQuantity) + .addToLimits("memory", driverMemoryLimitQuantity) + .addToLimits(maybeCpuLimitQuantity.toMap.asJava) + .endResources() + .build() + + val baseDriverPod = new PodBuilder(driverSpec.driverPod) + .editOrNewMetadata() + .withName(kubernetesDriverPodName) + .addToLabels(driverLabels.asJava) + .addToAnnotations(allDriverAnnotations.asJava) + .endMetadata() + .withNewSpec() + .withRestartPolicy("Never") + .withNodeSelector(nodeSelector.asJava) + .endSpec() + .build() + + val resolvedSparkConf = driverSpec.driverSparkConf.clone() + .setIfMissing(KUBERNETES_DRIVER_POD_NAME, kubernetesDriverPodName) + .set("spark.app.id", kubernetesAppId) + .set(KUBERNETES_EXECUTOR_POD_NAME_PREFIX, kubernetesResourceNamePrefix) + + driverSpec.copy( + driverPod = baseDriverPod, + driverSparkConf = resolvedSparkConf, + driverContainer = driverContainer) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverConfigurationStep.scala new file mode 100644 index 0000000000000..c99c0436cf25f --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverConfigurationStep.scala @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.submit.steps + +import org.apache.spark.deploy.k8s.submit.KubernetesDriverSpec + +/** + * Represents a step in preparing the Kubernetes driver. + */ +private[spark] trait DriverConfigurationStep { + + /** + * Apply some transformation to the previous state of the driver to add a new feature to it. + */ + def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverKubernetesCredentialsStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverKubernetesCredentialsStep.scala new file mode 100644 index 0000000000000..56856358dfe76 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverKubernetesCredentialsStep.scala @@ -0,0 +1,244 @@ +/* + * 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.deploy.k8s.submit.steps + +import java.io.File +import java.nio.charset.StandardCharsets + +import scala.collection.JavaConverters._ +import scala.language.implicitConversions + +import com.google.common.io.{BaseEncoding, Files} +import io.fabric8.kubernetes.api.model.{ContainerBuilder, PodBuilder, Secret, SecretBuilder} + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.submit.KubernetesDriverSpec + +/** + * Mounts Kubernetes credentials into the driver pod. The driver will use such mounted credentials + * to request executors. + */ +private[spark] class DriverKubernetesCredentialsStep( + submissionSparkConf: SparkConf, + kubernetesResourceNamePrefix: String) extends DriverConfigurationStep { + + private val maybeMountedOAuthTokenFile = submissionSparkConf.getOption( + s"$KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$OAUTH_TOKEN_FILE_CONF_SUFFIX") + private val maybeMountedClientKeyFile = submissionSparkConf.getOption( + s"$KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX") + private val maybeMountedClientCertFile = submissionSparkConf.getOption( + s"$KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX") + private val maybeMountedCaCertFile = submissionSparkConf.getOption( + s"$KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX") + private val driverServiceAccount = submissionSparkConf.get(KUBERNETES_SERVICE_ACCOUNT_NAME) + + override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { + val driverSparkConf = driverSpec.driverSparkConf.clone() + + val oauthTokenBase64 = submissionSparkConf + .getOption(s"$KUBERNETES_AUTH_DRIVER_CONF_PREFIX.$OAUTH_TOKEN_CONF_SUFFIX") + .map { token => + BaseEncoding.base64().encode(token.getBytes(StandardCharsets.UTF_8)) + } + val caCertDataBase64 = safeFileConfToBase64( + s"$KUBERNETES_AUTH_DRIVER_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX", + "Driver CA cert file provided at %s does not exist or is not a file.") + val clientKeyDataBase64 = safeFileConfToBase64( + s"$KUBERNETES_AUTH_DRIVER_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX", + "Driver client key file provided at %s does not exist or is not a file.") + val clientCertDataBase64 = safeFileConfToBase64( + s"$KUBERNETES_AUTH_DRIVER_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX", + "Driver client cert file provided at %s does not exist or is not a file.") + + val driverSparkConfWithCredentialsLocations = setDriverPodKubernetesCredentialLocations( + driverSparkConf, + oauthTokenBase64, + caCertDataBase64, + clientKeyDataBase64, + clientCertDataBase64) + + val kubernetesCredentialsSecret = createCredentialsSecret( + oauthTokenBase64, + caCertDataBase64, + clientKeyDataBase64, + clientCertDataBase64) + + val driverPodWithMountedKubernetesCredentials = kubernetesCredentialsSecret.map { secret => + new PodBuilder(driverSpec.driverPod) + .editOrNewSpec() + .addNewVolume() + .withName(DRIVER_CREDENTIALS_SECRET_VOLUME_NAME) + .withNewSecret().withSecretName(secret.getMetadata.getName).endSecret() + .endVolume() + .endSpec() + .build() + }.getOrElse( + driverServiceAccount.map { account => + new PodBuilder(driverSpec.driverPod) + .editOrNewSpec() + .withServiceAccount(account) + .withServiceAccountName(account) + .endSpec() + .build() + }.getOrElse(driverSpec.driverPod) + ) + + val driverContainerWithMountedSecretVolume = kubernetesCredentialsSecret.map { secret => + new ContainerBuilder(driverSpec.driverContainer) + .addNewVolumeMount() + .withName(DRIVER_CREDENTIALS_SECRET_VOLUME_NAME) + .withMountPath(DRIVER_CREDENTIALS_SECRETS_BASE_DIR) + .endVolumeMount() + .build() + }.getOrElse(driverSpec.driverContainer) + + driverSpec.copy( + driverPod = driverPodWithMountedKubernetesCredentials, + otherKubernetesResources = + driverSpec.otherKubernetesResources ++ kubernetesCredentialsSecret.toSeq, + driverSparkConf = driverSparkConfWithCredentialsLocations, + driverContainer = driverContainerWithMountedSecretVolume) + } + + private def createCredentialsSecret( + driverOAuthTokenBase64: Option[String], + driverCaCertDataBase64: Option[String], + driverClientKeyDataBase64: Option[String], + driverClientCertDataBase64: Option[String]): Option[Secret] = { + val allSecretData = + resolveSecretData( + maybeMountedClientKeyFile, + driverClientKeyDataBase64, + DRIVER_CREDENTIALS_CLIENT_KEY_SECRET_NAME) ++ + resolveSecretData( + maybeMountedClientCertFile, + driverClientCertDataBase64, + DRIVER_CREDENTIALS_CLIENT_CERT_SECRET_NAME) ++ + resolveSecretData( + maybeMountedCaCertFile, + driverCaCertDataBase64, + DRIVER_CREDENTIALS_CA_CERT_SECRET_NAME) ++ + resolveSecretData( + maybeMountedOAuthTokenFile, + driverOAuthTokenBase64, + DRIVER_CREDENTIALS_OAUTH_TOKEN_SECRET_NAME) + + if (allSecretData.isEmpty) { + None + } else { + Some(new SecretBuilder() + .withNewMetadata() + .withName(s"$kubernetesResourceNamePrefix-kubernetes-credentials") + .endMetadata() + .withData(allSecretData.asJava) + .build()) + } + } + + private def setDriverPodKubernetesCredentialLocations( + driverSparkConf: SparkConf, + driverOauthTokenBase64: Option[String], + driverCaCertDataBase64: Option[String], + driverClientKeyDataBase64: Option[String], + driverClientCertDataBase64: Option[String]): SparkConf = { + val resolvedMountedOAuthTokenFile = resolveSecretLocation( + maybeMountedOAuthTokenFile, + driverOauthTokenBase64, + DRIVER_CREDENTIALS_OAUTH_TOKEN_PATH) + val resolvedMountedClientKeyFile = resolveSecretLocation( + maybeMountedClientKeyFile, + driverClientKeyDataBase64, + DRIVER_CREDENTIALS_CLIENT_KEY_PATH) + val resolvedMountedClientCertFile = resolveSecretLocation( + maybeMountedClientCertFile, + driverClientCertDataBase64, + DRIVER_CREDENTIALS_CLIENT_CERT_PATH) + val resolvedMountedCaCertFile = resolveSecretLocation( + maybeMountedCaCertFile, + driverCaCertDataBase64, + DRIVER_CREDENTIALS_CA_CERT_PATH) + + val sparkConfWithCredentialLocations = driverSparkConf + .setOption( + s"$KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX", + resolvedMountedCaCertFile) + .setOption( + s"$KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX", + resolvedMountedClientKeyFile) + .setOption( + s"$KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX", + resolvedMountedClientCertFile) + .setOption( + s"$KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$OAUTH_TOKEN_FILE_CONF_SUFFIX", + resolvedMountedOAuthTokenFile) + + // Redact all OAuth token values + sparkConfWithCredentialLocations + .getAll + .filter(_._1.endsWith(OAUTH_TOKEN_CONF_SUFFIX)).map(_._1) + .foreach { + sparkConfWithCredentialLocations.set(_, "") + } + sparkConfWithCredentialLocations + } + + private def safeFileConfToBase64( + conf: String, + fileNotFoundFormatString: String): Option[String] = { + submissionSparkConf.getOption(conf) + .map(new File(_)) + .map { file => + require(file.isFile, String.format(fileNotFoundFormatString, file.getAbsolutePath)) + BaseEncoding.base64().encode(Files.toByteArray(file)) + } + } + + private def resolveSecretLocation( + mountedUserSpecified: Option[String], + valueMountedFromSubmitter: Option[String], + mountedCanonicalLocation: String): Option[String] = { + mountedUserSpecified.orElse(valueMountedFromSubmitter.map( _ => { + mountedCanonicalLocation + })) + } + + private def resolveSecretData( + mountedUserSpecified: Option[String], + valueMountedFromSubmitter: Option[String], + secretName: String): Map[String, String] = { + mountedUserSpecified.map { _ => Map.empty[String, String]} + .getOrElse { + valueMountedFromSubmitter.map { valueBase64 => + Map(secretName -> valueBase64) + }.getOrElse(Map.empty[String, String]) + } + } + + private implicit def augmentSparkConf(sparkConf: SparkConf): OptionSettableSparkConf = { + new OptionSettableSparkConf(sparkConf) + } +} + +private class OptionSettableSparkConf(sparkConf: SparkConf) { + def setOption(configEntry: String, option: Option[String]): SparkConf = { + option.map( opt => { + sparkConf.set(configEntry, opt) + }).getOrElse(sparkConf) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverServiceBootstrapStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverServiceBootstrapStep.scala new file mode 100644 index 0000000000000..5eb98a9ac9e5a --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverServiceBootstrapStep.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.deploy.k8s.submit.steps + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model.ServiceBuilder + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.submit.KubernetesDriverSpec +import org.apache.spark.internal.Logging +import org.apache.spark.util.Clock + +/** + * Allows the driver to be reachable by executor pods through a headless service. The service's + * ports should correspond to the ports that the executor will reach the pod at for RPC. + */ +private[spark] class DriverServiceBootstrapStep( + kubernetesResourceNamePrefix: String, + driverLabels: Map[String, String], + submissionSparkConf: SparkConf, + clock: Clock) extends DriverConfigurationStep with Logging { + import DriverServiceBootstrapStep._ + + override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { + require(submissionSparkConf.getOption(DRIVER_BIND_ADDRESS_KEY).isEmpty, + s"$DRIVER_BIND_ADDRESS_KEY is not supported in Kubernetes mode, as the driver's bind" + + " address is managed and set to the driver pod's IP address.") + require(submissionSparkConf.getOption(DRIVER_HOST_KEY).isEmpty, + s"$DRIVER_HOST_KEY is not supported in Kubernetes mode, as the driver's hostname will be" + + " managed via a Kubernetes service.") + + val preferredServiceName = s"$kubernetesResourceNamePrefix$DRIVER_SVC_POSTFIX" + val resolvedServiceName = if (preferredServiceName.length <= MAX_SERVICE_NAME_LENGTH) { + preferredServiceName + } else { + val randomServiceId = clock.getTimeMillis() + val shorterServiceName = s"spark-$randomServiceId$DRIVER_SVC_POSTFIX" + logWarning(s"Driver's hostname would preferably be $preferredServiceName, but this is" + + s" too long (must be <= 63 characters). Falling back to use $shorterServiceName" + + " as the driver service's name.") + shorterServiceName + } + + val driverPort = submissionSparkConf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT) + val driverBlockManagerPort = submissionSparkConf.getInt( + org.apache.spark.internal.config.DRIVER_BLOCK_MANAGER_PORT.key, DEFAULT_BLOCKMANAGER_PORT) + val driverService = new ServiceBuilder() + .withNewMetadata() + .withName(resolvedServiceName) + .endMetadata() + .withNewSpec() + .withClusterIP("None") + .withSelector(driverLabels.asJava) + .addNewPort() + .withName(DRIVER_PORT_NAME) + .withPort(driverPort) + .withNewTargetPort(driverPort) + .endPort() + .addNewPort() + .withName(BLOCK_MANAGER_PORT_NAME) + .withPort(driverBlockManagerPort) + .withNewTargetPort(driverBlockManagerPort) + .endPort() + .endSpec() + .build() + + val namespace = submissionSparkConf.get(KUBERNETES_NAMESPACE) + val driverHostname = s"${driverService.getMetadata.getName}.$namespace.svc.cluster.local" + val resolvedSparkConf = driverSpec.driverSparkConf.clone() + .set(org.apache.spark.internal.config.DRIVER_HOST_ADDRESS, driverHostname) + .set("spark.driver.port", driverPort.toString) + .set( + org.apache.spark.internal.config.DRIVER_BLOCK_MANAGER_PORT, driverBlockManagerPort) + + driverSpec.copy( + driverSparkConf = resolvedSparkConf, + otherKubernetesResources = driverSpec.otherKubernetesResources ++ Seq(driverService)) + } +} + +private[spark] object DriverServiceBootstrapStep { + val DRIVER_BIND_ADDRESS_KEY = org.apache.spark.internal.config.DRIVER_BIND_ADDRESS.key + val DRIVER_HOST_KEY = org.apache.spark.internal.config.DRIVER_HOST_ADDRESS.key + val DRIVER_SVC_POSTFIX = "-driver-svc" + val MAX_SERVICE_NAME_LENGTH = 63 +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala index 68ca6a7622171..31d65b95e21da 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala @@ -45,7 +45,7 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit val kubernetesClient = SparkKubernetesClientFactory.createKubernetesClient( KUBERNETES_MASTER_INTERNAL_URL, Some(sparkConf.get(KUBERNETES_NAMESPACE)), - APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX, + KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX, sparkConf, Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_TOKEN_PATH)), Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_CA_CRT_PATH))) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala new file mode 100644 index 0000000000000..bf4ec04893204 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala @@ -0,0 +1,234 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.submit + +import scala.collection.JavaConverters._ + +import com.google.common.collect.Iterables +import io.fabric8.kubernetes.api.model._ +import io.fabric8.kubernetes.client.{KubernetesClient, Watch} +import io.fabric8.kubernetes.client.dsl.{MixedOperation, NamespaceListVisitFromServerGetDeleteRecreateWaitApplicable, PodResource} +import org.mockito.{ArgumentCaptor, Mock, MockitoAnnotations} +import org.mockito.Mockito.{doReturn, verify, when} +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer +import org.scalatest.BeforeAndAfter +import org.scalatest.mockito.MockitoSugar._ + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.submit.steps.DriverConfigurationStep + +class ClientSuite extends SparkFunSuite with BeforeAndAfter { + + private val DRIVER_POD_UID = "pod-id" + private val DRIVER_POD_API_VERSION = "v1" + private val DRIVER_POD_KIND = "pod" + + private type ResourceList = NamespaceListVisitFromServerGetDeleteRecreateWaitApplicable[ + HasMetadata, Boolean] + private type Pods = MixedOperation[Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]] + + @Mock + private var kubernetesClient: KubernetesClient = _ + + @Mock + private var podOperations: Pods = _ + + @Mock + private var namedPods: PodResource[Pod, DoneablePod] = _ + + @Mock + private var loggingPodStatusWatcher: LoggingPodStatusWatcher = _ + + @Mock + private var resourceList: ResourceList = _ + + private val submissionSteps = Seq(FirstTestConfigurationStep, SecondTestConfigurationStep) + private var createdPodArgumentCaptor: ArgumentCaptor[Pod] = _ + private var createdResourcesArgumentCaptor: ArgumentCaptor[HasMetadata] = _ + + before { + MockitoAnnotations.initMocks(this) + when(kubernetesClient.pods()).thenReturn(podOperations) + when(podOperations.withName(FirstTestConfigurationStep.podName)).thenReturn(namedPods) + + createdPodArgumentCaptor = ArgumentCaptor.forClass(classOf[Pod]) + createdResourcesArgumentCaptor = ArgumentCaptor.forClass(classOf[HasMetadata]) + when(podOperations.create(createdPodArgumentCaptor.capture())).thenAnswer(new Answer[Pod] { + override def answer(invocation: InvocationOnMock): Pod = { + new PodBuilder(invocation.getArgumentAt(0, classOf[Pod])) + .editMetadata() + .withUid(DRIVER_POD_UID) + .endMetadata() + .withApiVersion(DRIVER_POD_API_VERSION) + .withKind(DRIVER_POD_KIND) + .build() + } + }) + when(podOperations.withName(FirstTestConfigurationStep.podName)).thenReturn(namedPods) + when(namedPods.watch(loggingPodStatusWatcher)).thenReturn(mock[Watch]) + doReturn(resourceList) + .when(kubernetesClient) + .resourceList(createdResourcesArgumentCaptor.capture()) + } + + test("The client should configure the pod with the submission steps.") { + val submissionClient = new Client( + submissionSteps, + new SparkConf(false), + kubernetesClient, + false, + "spark", + loggingPodStatusWatcher) + submissionClient.run() + val createdPod = createdPodArgumentCaptor.getValue + assert(createdPod.getMetadata.getName === FirstTestConfigurationStep.podName) + assert(createdPod.getMetadata.getLabels.asScala === + Map(FirstTestConfigurationStep.labelKey -> FirstTestConfigurationStep.labelValue)) + assert(createdPod.getMetadata.getAnnotations.asScala === + Map(SecondTestConfigurationStep.annotationKey -> + SecondTestConfigurationStep.annotationValue)) + assert(createdPod.getSpec.getContainers.size() === 1) + assert(createdPod.getSpec.getContainers.get(0).getName === + SecondTestConfigurationStep.containerName) + } + + test("The client should create the secondary Kubernetes resources.") { + val submissionClient = new Client( + submissionSteps, + new SparkConf(false), + kubernetesClient, + false, + "spark", + loggingPodStatusWatcher) + submissionClient.run() + val createdPod = createdPodArgumentCaptor.getValue + val otherCreatedResources = createdResourcesArgumentCaptor.getAllValues + assert(otherCreatedResources.size === 1) + val createdResource = Iterables.getOnlyElement(otherCreatedResources).asInstanceOf[Secret] + assert(createdResource.getMetadata.getName === FirstTestConfigurationStep.secretName) + assert(createdResource.getData.asScala === + Map(FirstTestConfigurationStep.secretKey -> FirstTestConfigurationStep.secretData)) + val ownerReference = Iterables.getOnlyElement(createdResource.getMetadata.getOwnerReferences) + assert(ownerReference.getName === createdPod.getMetadata.getName) + assert(ownerReference.getKind === DRIVER_POD_KIND) + assert(ownerReference.getUid === DRIVER_POD_UID) + assert(ownerReference.getApiVersion === DRIVER_POD_API_VERSION) + } + + test("The client should attach the driver container with the appropriate JVM options.") { + val sparkConf = new SparkConf(false) + .set("spark.logConf", "true") + .set( + org.apache.spark.internal.config.DRIVER_JAVA_OPTIONS, + "-XX:+HeapDumpOnOutOfMemoryError -XX:+PrintGCDetails") + val submissionClient = new Client( + submissionSteps, + sparkConf, + kubernetesClient, + false, + "spark", + loggingPodStatusWatcher) + submissionClient.run() + val createdPod = createdPodArgumentCaptor.getValue + val driverContainer = Iterables.getOnlyElement(createdPod.getSpec.getContainers) + assert(driverContainer.getName === SecondTestConfigurationStep.containerName) + val driverJvmOptsEnvs = driverContainer.getEnv.asScala.filter { env => + env.getName.startsWith(ENV_JAVA_OPT_PREFIX) + }.sortBy(_.getName) + assert(driverJvmOptsEnvs.size === 4) + + val expectedJvmOptsValues = Seq( + "-Dspark.logConf=true", + s"-D${SecondTestConfigurationStep.sparkConfKey}=" + + s"${SecondTestConfigurationStep.sparkConfValue}", + "-XX:+HeapDumpOnOutOfMemoryError", + "-XX:+PrintGCDetails") + driverJvmOptsEnvs.zip(expectedJvmOptsValues).zipWithIndex.foreach { + case ((resolvedEnv, expectedJvmOpt), index) => + assert(resolvedEnv.getName === s"$ENV_JAVA_OPT_PREFIX$index") + assert(resolvedEnv.getValue === expectedJvmOpt) + } + } + + test("Waiting for app completion should stall on the watcher") { + val submissionClient = new Client( + submissionSteps, + new SparkConf(false), + kubernetesClient, + true, + "spark", + loggingPodStatusWatcher) + submissionClient.run() + verify(loggingPodStatusWatcher).awaitCompletion() + } + +} + +private object FirstTestConfigurationStep extends DriverConfigurationStep { + + val podName = "test-pod" + val secretName = "test-secret" + val labelKey = "first-submit" + val labelValue = "true" + val secretKey = "secretKey" + val secretData = "secretData" + + override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { + val modifiedPod = new PodBuilder(driverSpec.driverPod) + .editMetadata() + .withName(podName) + .addToLabels(labelKey, labelValue) + .endMetadata() + .build() + val additionalResource = new SecretBuilder() + .withNewMetadata() + .withName(secretName) + .endMetadata() + .addToData(secretKey, secretData) + .build() + driverSpec.copy( + driverPod = modifiedPod, + otherKubernetesResources = driverSpec.otherKubernetesResources ++ Seq(additionalResource)) + } +} + +private object SecondTestConfigurationStep extends DriverConfigurationStep { + + val annotationKey = "second-submit" + val annotationValue = "submitted" + val sparkConfKey = "spark.custom-conf" + val sparkConfValue = "custom-conf-value" + val containerName = "driverContainer" + + override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { + val modifiedPod = new PodBuilder(driverSpec.driverPod) + .editMetadata() + .addToAnnotations(annotationKey, annotationValue) + .endMetadata() + .build() + val resolvedSparkConf = driverSpec.driverSparkConf.clone().set(sparkConfKey, sparkConfValue) + val modifiedContainer = new ContainerBuilder(driverSpec.driverContainer) + .withName(containerName) + .build() + driverSpec.copy( + driverPod = modifiedPod, + driverSparkConf = resolvedSparkConf, + driverContainer = modifiedContainer) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala new file mode 100644 index 0000000000000..3a1603fd49b16 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.submit + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s.Config.DRIVER_DOCKER_IMAGE +import org.apache.spark.deploy.k8s.submit.steps.{BaseDriverConfigurationStep, DriverConfigurationStep, DriverKubernetesCredentialsStep, DriverServiceBootstrapStep} + +class DriverConfigurationStepsOrchestratorSuite extends SparkFunSuite { + + private val NAMESPACE = "default" + private val DRIVER_IMAGE = "driver-image" + private val APP_ID = "spark-app-id" + private val LAUNCH_TIME = 975256L + private val APP_NAME = "spark" + private val MAIN_CLASS = "org.apache.spark.examples.SparkPi" + private val APP_ARGS = Array("arg1", "arg2") + + test("Base submission steps without an init-container or python files.") { + val sparkConf = new SparkConf(false) + .set(DRIVER_DOCKER_IMAGE, DRIVER_IMAGE) + val mainAppResource = JavaMainAppResource("local:///var/apps/jars/main.jar") + val orchestrator = new DriverConfigurationStepsOrchestrator( + NAMESPACE, + APP_ID, + LAUNCH_TIME, + mainAppResource, + APP_NAME, + MAIN_CLASS, + APP_ARGS, + sparkConf) + validateStepTypes( + orchestrator, + classOf[BaseDriverConfigurationStep], + classOf[DriverServiceBootstrapStep], + classOf[DriverKubernetesCredentialsStep] + ) + } + + private def validateStepTypes( + orchestrator: DriverConfigurationStepsOrchestrator, + types: Class[_ <: DriverConfigurationStep]*): Unit = { + val steps = orchestrator.getAllConfigurationSteps() + assert(steps.size === types.size) + assert(steps.map(_.getClass) === types) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStepSuite.scala new file mode 100644 index 0000000000000..8af6f438c2ba0 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStepSuite.scala @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.submit.steps + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model.{ContainerBuilder, HasMetadata, PodBuilder} + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.submit.KubernetesDriverSpec + +class BaseDriverConfigurationStepSuite extends SparkFunSuite { + + private val APP_ID = "spark-app-id" + private val RESOURCE_NAME_PREFIX = "spark" + private val DRIVER_LABELS = Map("labelkey" -> "labelvalue") + private val DOCKER_IMAGE_PULL_POLICY = "IfNotPresent" + private val APP_NAME = "spark-test" + private val MAIN_CLASS = "org.apache.spark.examples.SparkPi" + private val APP_ARGS = Array("arg1", "arg2") + private val CUSTOM_ANNOTATION_KEY = "customAnnotation" + private val CUSTOM_ANNOTATION_VALUE = "customAnnotationValue" + private val DRIVER_CUSTOM_ENV_KEY1 = "customDriverEnv1" + private val DRIVER_CUSTOM_ENV_KEY2 = "customDriverEnv2" + + test("Set all possible configurations from the user.") { + val sparkConf = new SparkConf() + .set(KUBERNETES_DRIVER_POD_NAME, "spark-driver-pod") + .set(org.apache.spark.internal.config.DRIVER_CLASS_PATH, "/opt/spark/spark-examples.jar") + .set("spark.driver.cores", "2") + .set(KUBERNETES_DRIVER_LIMIT_CORES, "4") + .set(org.apache.spark.internal.config.DRIVER_MEMORY.key, "256M") + .set(KUBERNETES_DRIVER_MEMORY_OVERHEAD, 200L) + .set(DRIVER_DOCKER_IMAGE, "spark-driver:latest") + .set(s"$KUBERNETES_DRIVER_ANNOTATION_PREFIX$CUSTOM_ANNOTATION_KEY", CUSTOM_ANNOTATION_VALUE) + .set(s"$KUBERNETES_DRIVER_ENV_KEY$DRIVER_CUSTOM_ENV_KEY1", "customDriverEnv1") + .set(s"$KUBERNETES_DRIVER_ENV_KEY$DRIVER_CUSTOM_ENV_KEY2", "customDriverEnv2") + + val submissionStep = new BaseDriverConfigurationStep( + APP_ID, + RESOURCE_NAME_PREFIX, + DRIVER_LABELS, + DOCKER_IMAGE_PULL_POLICY, + APP_NAME, + MAIN_CLASS, + APP_ARGS, + sparkConf) + val basePod = new PodBuilder().withNewMetadata().endMetadata().withNewSpec().endSpec().build() + val baseDriverSpec = KubernetesDriverSpec( + driverPod = basePod, + driverContainer = new ContainerBuilder().build(), + driverSparkConf = new SparkConf(false), + otherKubernetesResources = Seq.empty[HasMetadata]) + val preparedDriverSpec = submissionStep.configureDriver(baseDriverSpec) + + assert(preparedDriverSpec.driverContainer.getName === DRIVER_CONTAINER_NAME) + assert(preparedDriverSpec.driverContainer.getImage === "spark-driver:latest") + assert(preparedDriverSpec.driverContainer.getImagePullPolicy === DOCKER_IMAGE_PULL_POLICY) + + assert(preparedDriverSpec.driverContainer.getEnv.size === 7) + val envs = preparedDriverSpec.driverContainer + .getEnv + .asScala + .map(env => (env.getName, env.getValue)) + .toMap + assert(envs(ENV_SUBMIT_EXTRA_CLASSPATH) === "/opt/spark/spark-examples.jar") + assert(envs(ENV_DRIVER_MEMORY) === "256M") + assert(envs(ENV_DRIVER_MAIN_CLASS) === MAIN_CLASS) + assert(envs(ENV_DRIVER_ARGS) === "arg1 arg2") + assert(envs(DRIVER_CUSTOM_ENV_KEY1) === "customDriverEnv1") + assert(envs(DRIVER_CUSTOM_ENV_KEY2) === "customDriverEnv2") + + assert(preparedDriverSpec.driverContainer.getEnv.asScala.exists(envVar => + envVar.getName.equals(ENV_DRIVER_BIND_ADDRESS) && + envVar.getValueFrom.getFieldRef.getApiVersion.equals("v1") && + envVar.getValueFrom.getFieldRef.getFieldPath.equals("status.podIP"))) + + val resourceRequirements = preparedDriverSpec.driverContainer.getResources + val requests = resourceRequirements.getRequests.asScala + assert(requests("cpu").getAmount === "2") + assert(requests("memory").getAmount === "256Mi") + val limits = resourceRequirements.getLimits.asScala + assert(limits("memory").getAmount === "456Mi") + assert(limits("cpu").getAmount === "4") + + val driverPodMetadata = preparedDriverSpec.driverPod.getMetadata + assert(driverPodMetadata.getName === "spark-driver-pod") + assert(driverPodMetadata.getLabels.asScala === DRIVER_LABELS) + val expectedAnnotations = Map( + CUSTOM_ANNOTATION_KEY -> CUSTOM_ANNOTATION_VALUE, + SPARK_APP_NAME_ANNOTATION -> APP_NAME) + assert(driverPodMetadata.getAnnotations.asScala === expectedAnnotations) + assert(preparedDriverSpec.driverPod.getSpec.getRestartPolicy === "Never") + + val resolvedSparkConf = preparedDriverSpec.driverSparkConf.getAll.toMap + val expectedSparkConf = Map( + KUBERNETES_DRIVER_POD_NAME.key -> "spark-driver-pod", + "spark.app.id" -> APP_ID, + KUBERNETES_EXECUTOR_POD_NAME_PREFIX.key -> RESOURCE_NAME_PREFIX) + assert(resolvedSparkConf === expectedSparkConf) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverKubernetesCredentialsStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverKubernetesCredentialsStepSuite.scala new file mode 100644 index 0000000000000..64553d25883bb --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverKubernetesCredentialsStepSuite.scala @@ -0,0 +1,153 @@ +/* + * 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.deploy.k8s.submit.steps + +import java.io.File + +import scala.collection.JavaConverters._ + +import com.google.common.base.Charsets +import com.google.common.io.{BaseEncoding, Files} +import io.fabric8.kubernetes.api.model.{ContainerBuilder, HasMetadata, PodBuilder, Secret} +import org.scalatest.BeforeAndAfter + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.submit.KubernetesDriverSpec +import org.apache.spark.util.Utils + +class DriverKubernetesCredentialsStepSuite extends SparkFunSuite with BeforeAndAfter { + + private val KUBERNETES_RESOURCE_NAME_PREFIX = "spark" + private var credentialsTempDirectory: File = _ + private val BASE_DRIVER_SPEC = new KubernetesDriverSpec( + driverPod = new PodBuilder().build(), + driverContainer = new ContainerBuilder().build(), + driverSparkConf = new SparkConf(false), + otherKubernetesResources = Seq.empty[HasMetadata]) + + before { + credentialsTempDirectory = Utils.createTempDir() + } + + after { + credentialsTempDirectory.delete() + } + + test("Don't set any credentials") { + val kubernetesCredentialsStep = new DriverKubernetesCredentialsStep( + new SparkConf(false), KUBERNETES_RESOURCE_NAME_PREFIX) + val preparedDriverSpec = kubernetesCredentialsStep.configureDriver(BASE_DRIVER_SPEC) + assert(preparedDriverSpec.driverPod === BASE_DRIVER_SPEC.driverPod) + assert(preparedDriverSpec.driverContainer === BASE_DRIVER_SPEC.driverContainer) + assert(preparedDriverSpec.otherKubernetesResources.isEmpty) + assert(preparedDriverSpec.driverSparkConf.getAll.isEmpty) + } + + test("Only set credentials that are manually mounted.") { + val submissionSparkConf = new SparkConf(false) + .set( + s"$KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$OAUTH_TOKEN_FILE_CONF_SUFFIX", + "/mnt/secrets/my-token.txt") + .set( + s"$KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX", + "/mnt/secrets/my-key.pem") + .set( + s"$KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX", + "/mnt/secrets/my-cert.pem") + .set( + s"$KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX", + "/mnt/secrets/my-ca.pem") + + val kubernetesCredentialsStep = new DriverKubernetesCredentialsStep( + submissionSparkConf, KUBERNETES_RESOURCE_NAME_PREFIX) + val preparedDriverSpec = kubernetesCredentialsStep.configureDriver(BASE_DRIVER_SPEC) + assert(preparedDriverSpec.driverPod === BASE_DRIVER_SPEC.driverPod) + assert(preparedDriverSpec.driverContainer === BASE_DRIVER_SPEC.driverContainer) + assert(preparedDriverSpec.otherKubernetesResources.isEmpty) + assert(preparedDriverSpec.driverSparkConf.getAll.toMap === submissionSparkConf.getAll.toMap) + } + + test("Mount credentials from the submission client as a secret.") { + val caCertFile = writeCredentials("ca.pem", "ca-cert") + val clientKeyFile = writeCredentials("key.pem", "key") + val clientCertFile = writeCredentials("cert.pem", "cert") + val submissionSparkConf = new SparkConf(false) + .set( + s"$KUBERNETES_AUTH_DRIVER_CONF_PREFIX.$OAUTH_TOKEN_CONF_SUFFIX", + "token") + .set( + s"$KUBERNETES_AUTH_DRIVER_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX", + clientKeyFile.getAbsolutePath) + .set( + s"$KUBERNETES_AUTH_DRIVER_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX", + clientCertFile.getAbsolutePath) + .set( + s"$KUBERNETES_AUTH_DRIVER_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX", + caCertFile.getAbsolutePath) + val kubernetesCredentialsStep = new DriverKubernetesCredentialsStep( + submissionSparkConf, KUBERNETES_RESOURCE_NAME_PREFIX) + val preparedDriverSpec = kubernetesCredentialsStep.configureDriver( + BASE_DRIVER_SPEC.copy(driverSparkConf = submissionSparkConf)) + val expectedSparkConf = Map( + s"$KUBERNETES_AUTH_DRIVER_CONF_PREFIX.$OAUTH_TOKEN_CONF_SUFFIX" -> "", + s"$KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$OAUTH_TOKEN_FILE_CONF_SUFFIX" -> + DRIVER_CREDENTIALS_OAUTH_TOKEN_PATH, + s"$KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX" -> + DRIVER_CREDENTIALS_CLIENT_KEY_PATH, + s"$KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX" -> + DRIVER_CREDENTIALS_CLIENT_CERT_PATH, + s"$KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX" -> + DRIVER_CREDENTIALS_CA_CERT_PATH, + s"$KUBERNETES_AUTH_DRIVER_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX" -> + clientKeyFile.getAbsolutePath, + s"$KUBERNETES_AUTH_DRIVER_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX" -> + clientCertFile.getAbsolutePath, + s"$KUBERNETES_AUTH_DRIVER_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX" -> + caCertFile.getAbsolutePath) + assert(preparedDriverSpec.driverSparkConf.getAll.toMap === expectedSparkConf) + assert(preparedDriverSpec.otherKubernetesResources.size === 1) + val credentialsSecret = preparedDriverSpec.otherKubernetesResources.head.asInstanceOf[Secret] + assert(credentialsSecret.getMetadata.getName === + s"$KUBERNETES_RESOURCE_NAME_PREFIX-kubernetes-credentials") + val decodedSecretData = credentialsSecret.getData.asScala.map { data => + (data._1, new String(BaseEncoding.base64().decode(data._2), Charsets.UTF_8)) + } + val expectedSecretData = Map( + DRIVER_CREDENTIALS_CA_CERT_SECRET_NAME -> "ca-cert", + DRIVER_CREDENTIALS_OAUTH_TOKEN_SECRET_NAME -> "token", + DRIVER_CREDENTIALS_CLIENT_KEY_SECRET_NAME -> "key", + DRIVER_CREDENTIALS_CLIENT_CERT_SECRET_NAME -> "cert") + assert(decodedSecretData === expectedSecretData) + val driverPodVolumes = preparedDriverSpec.driverPod.getSpec.getVolumes.asScala + assert(driverPodVolumes.size === 1) + assert(driverPodVolumes.head.getName === DRIVER_CREDENTIALS_SECRET_VOLUME_NAME) + assert(driverPodVolumes.head.getSecret != null) + assert(driverPodVolumes.head.getSecret.getSecretName === credentialsSecret.getMetadata.getName) + val driverContainerVolumeMount = preparedDriverSpec.driverContainer.getVolumeMounts.asScala + assert(driverContainerVolumeMount.size === 1) + assert(driverContainerVolumeMount.head.getName === DRIVER_CREDENTIALS_SECRET_VOLUME_NAME) + assert(driverContainerVolumeMount.head.getMountPath === DRIVER_CREDENTIALS_SECRETS_BASE_DIR) + } + + private def writeCredentials(credentialsFileName: String, credentialsContents: String): File = { + val credentialsFile = new File(credentialsTempDirectory, credentialsFileName) + Files.write(credentialsContents, credentialsFile, Charsets.UTF_8) + credentialsFile + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverServiceBootstrapStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverServiceBootstrapStepSuite.scala new file mode 100644 index 0000000000000..006ce2668f8a0 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DriverServiceBootstrapStepSuite.scala @@ -0,0 +1,180 @@ +/* + * 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.deploy.k8s.submit.steps + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model.Service +import org.mockito.{Mock, MockitoAnnotations} +import org.mockito.Mockito.when +import org.scalatest.BeforeAndAfter + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.submit.KubernetesDriverSpec +import org.apache.spark.util.Clock + +class DriverServiceBootstrapStepSuite extends SparkFunSuite with BeforeAndAfter { + + private val SHORT_RESOURCE_NAME_PREFIX = + "a" * (DriverServiceBootstrapStep.MAX_SERVICE_NAME_LENGTH - + DriverServiceBootstrapStep.DRIVER_SVC_POSTFIX.length) + + private val LONG_RESOURCE_NAME_PREFIX = + "a" * (DriverServiceBootstrapStep.MAX_SERVICE_NAME_LENGTH - + DriverServiceBootstrapStep.DRIVER_SVC_POSTFIX.length + 1) + private val DRIVER_LABELS = Map( + "label1key" -> "label1value", + "label2key" -> "label2value") + + @Mock + private var clock: Clock = _ + + private var sparkConf: SparkConf = _ + + before { + MockitoAnnotations.initMocks(this) + sparkConf = new SparkConf(false) + } + + test("Headless service has a port for the driver RPC and the block manager.") { + val configurationStep = new DriverServiceBootstrapStep( + SHORT_RESOURCE_NAME_PREFIX, + DRIVER_LABELS, + sparkConf + .set("spark.driver.port", "9000") + .set(org.apache.spark.internal.config.DRIVER_BLOCK_MANAGER_PORT, 8080), + clock) + val baseDriverSpec = KubernetesDriverSpec.initialSpec(sparkConf.clone()) + val resolvedDriverSpec = configurationStep.configureDriver(baseDriverSpec) + assert(resolvedDriverSpec.otherKubernetesResources.size === 1) + assert(resolvedDriverSpec.otherKubernetesResources.head.isInstanceOf[Service]) + val driverService = resolvedDriverSpec.otherKubernetesResources.head.asInstanceOf[Service] + verifyService( + 9000, + 8080, + s"$SHORT_RESOURCE_NAME_PREFIX${DriverServiceBootstrapStep.DRIVER_SVC_POSTFIX}", + driverService) + } + + test("Hostname and ports are set according to the service name.") { + val configurationStep = new DriverServiceBootstrapStep( + SHORT_RESOURCE_NAME_PREFIX, + DRIVER_LABELS, + sparkConf + .set("spark.driver.port", "9000") + .set(org.apache.spark.internal.config.DRIVER_BLOCK_MANAGER_PORT, 8080) + .set(KUBERNETES_NAMESPACE, "my-namespace"), + clock) + val baseDriverSpec = KubernetesDriverSpec.initialSpec(sparkConf.clone()) + val resolvedDriverSpec = configurationStep.configureDriver(baseDriverSpec) + val expectedServiceName = SHORT_RESOURCE_NAME_PREFIX + + DriverServiceBootstrapStep.DRIVER_SVC_POSTFIX + val expectedHostName = s"$expectedServiceName.my-namespace.svc.cluster.local" + verifySparkConfHostNames(resolvedDriverSpec.driverSparkConf, expectedHostName) + } + + test("Ports should resolve to defaults in SparkConf and in the service.") { + val configurationStep = new DriverServiceBootstrapStep( + SHORT_RESOURCE_NAME_PREFIX, + DRIVER_LABELS, + sparkConf, + clock) + val baseDriverSpec = KubernetesDriverSpec.initialSpec(sparkConf.clone()) + val resolvedDriverSpec = configurationStep.configureDriver(baseDriverSpec) + verifyService( + DEFAULT_DRIVER_PORT, + DEFAULT_BLOCKMANAGER_PORT, + s"$SHORT_RESOURCE_NAME_PREFIX${DriverServiceBootstrapStep.DRIVER_SVC_POSTFIX}", + resolvedDriverSpec.otherKubernetesResources.head.asInstanceOf[Service]) + assert(resolvedDriverSpec.driverSparkConf.get("spark.driver.port") === + DEFAULT_DRIVER_PORT.toString) + assert(resolvedDriverSpec.driverSparkConf.get( + org.apache.spark.internal.config.DRIVER_BLOCK_MANAGER_PORT) === DEFAULT_BLOCKMANAGER_PORT) + } + + test("Long prefixes should switch to using a generated name.") { + val configurationStep = new DriverServiceBootstrapStep( + LONG_RESOURCE_NAME_PREFIX, + DRIVER_LABELS, + sparkConf.set(KUBERNETES_NAMESPACE, "my-namespace"), + clock) + when(clock.getTimeMillis()).thenReturn(10000) + val baseDriverSpec = KubernetesDriverSpec.initialSpec(sparkConf.clone()) + val resolvedDriverSpec = configurationStep.configureDriver(baseDriverSpec) + val driverService = resolvedDriverSpec.otherKubernetesResources.head.asInstanceOf[Service] + val expectedServiceName = s"spark-10000${DriverServiceBootstrapStep.DRIVER_SVC_POSTFIX}" + assert(driverService.getMetadata.getName === expectedServiceName) + val expectedHostName = s"$expectedServiceName.my-namespace.svc.cluster.local" + verifySparkConfHostNames(resolvedDriverSpec.driverSparkConf, expectedHostName) + } + + test("Disallow bind address and driver host to be set explicitly.") { + val configurationStep = new DriverServiceBootstrapStep( + LONG_RESOURCE_NAME_PREFIX, + DRIVER_LABELS, + sparkConf.set(org.apache.spark.internal.config.DRIVER_BIND_ADDRESS, "host"), + clock) + try { + configurationStep.configureDriver(KubernetesDriverSpec.initialSpec(sparkConf)) + fail("The driver bind address should not be allowed.") + } catch { + case e: Throwable => + assert(e.getMessage === + s"requirement failed: ${DriverServiceBootstrapStep.DRIVER_BIND_ADDRESS_KEY} is" + + " not supported in Kubernetes mode, as the driver's bind address is managed" + + " and set to the driver pod's IP address.") + } + sparkConf.remove(org.apache.spark.internal.config.DRIVER_BIND_ADDRESS) + sparkConf.set(org.apache.spark.internal.config.DRIVER_HOST_ADDRESS, "host") + try { + configurationStep.configureDriver(KubernetesDriverSpec.initialSpec(sparkConf)) + fail("The driver host address should not be allowed.") + } catch { + case e: Throwable => + assert(e.getMessage === + s"requirement failed: ${DriverServiceBootstrapStep.DRIVER_HOST_KEY} is" + + " not supported in Kubernetes mode, as the driver's hostname will be managed via" + + " a Kubernetes service.") + } + } + + private def verifyService( + driverPort: Int, + blockManagerPort: Int, + expectedServiceName: String, + service: Service): Unit = { + assert(service.getMetadata.getName === expectedServiceName) + assert(service.getSpec.getClusterIP === "None") + assert(service.getSpec.getSelector.asScala === DRIVER_LABELS) + assert(service.getSpec.getPorts.size() === 2) + val driverServicePorts = service.getSpec.getPorts.asScala + assert(driverServicePorts.head.getName === DRIVER_PORT_NAME) + assert(driverServicePorts.head.getPort.intValue() === driverPort) + assert(driverServicePorts.head.getTargetPort.getIntVal === driverPort) + assert(driverServicePorts(1).getName === BLOCK_MANAGER_PORT_NAME) + assert(driverServicePorts(1).getPort.intValue() === blockManagerPort) + assert(driverServicePorts(1).getTargetPort.getIntVal === blockManagerPort) + } + + private def verifySparkConfHostNames( + driverSparkConf: SparkConf, expectedHostName: String): Unit = { + assert(driverSparkConf.get( + org.apache.spark.internal.config.DRIVER_HOST_ADDRESS) === expectedHostName) + } +} diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/driver/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/driver/Dockerfile new file mode 100644 index 0000000000000..6b9ec4d193441 --- /dev/null +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/driver/Dockerfile @@ -0,0 +1,31 @@ +# +# 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. +# + +FROM spark-base + +# If this docker file is being used in the context of building your images from a Spark distribution, the docker build +# command should be invoked from the top level directory of the Spark distribution. E.g.: +# docker build -t spark-driver:latest -f dockerfiles/driver/Dockerfile . + +COPY examples /opt/spark/examples + +CMD SPARK_CLASSPATH="${SPARK_HOME}/jars/*" && \ + env | grep SPARK_JAVA_OPT_ | sed 's/[^=]*=\(.*\)/\1/g' > /tmp/java_opts.txt && \ + readarray -t SPARK_DRIVER_JAVA_OPTS < /tmp/java_opts.txt && \ + if ! [ -z ${SPARK_MOUNTED_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_MOUNTED_CLASSPATH:$SPARK_CLASSPATH"; fi && \ + if ! [ -z ${SPARK_SUBMIT_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_SUBMIT_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \ + ${JAVA_HOME}/bin/java "${SPARK_DRIVER_JAVA_OPTS[@]}" -cp "$SPARK_CLASSPATH" -Xms$SPARK_DRIVER_MEMORY -Xmx$SPARK_DRIVER_MEMORY -Dspark.driver.bindAddress=$SPARK_DRIVER_BIND_ADDRESS $SPARK_DRIVER_CLASS $SPARK_DRIVER_ARGS \ No newline at end of file diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/executor/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/executor/Dockerfile new file mode 100644 index 0000000000000..878120dcbda0b --- /dev/null +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/executor/Dockerfile @@ -0,0 +1,31 @@ +# +# 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. +# + +FROM spark-base + +# If this docker file is being used in the context of building your images from a Spark distribution, the docker build +# command should be invoked from the top level directory of the Spark distribution. E.g.: +# docker build -t spark-executor:latest -f dockerfiles/executor/Dockerfile . + +COPY examples /opt/spark/examples + +CMD SPARK_CLASSPATH="${SPARK_HOME}/jars/*" && \ + env | grep SPARK_JAVA_OPT_ | sed 's/[^=]*=\(.*\)/\1/g' > /tmp/java_opts.txt && \ + readarray -t SPARK_EXECUTOR_JAVA_OPTS < /tmp/java_opts.txt && \ + if ! [ -z ${SPARK_MOUNTED_CLASSPATH}+x} ]; then SPARK_CLASSPATH="$SPARK_MOUNTED_CLASSPATH:$SPARK_CLASSPATH"; fi && \ + if ! [ -z ${SPARK_EXECUTOR_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_EXECUTOR_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \ + ${JAVA_HOME}/bin/java "${SPARK_EXECUTOR_JAVA_OPTS[@]}" -Dspark.executor.port=$SPARK_EXECUTOR_PORT -Xms$SPARK_EXECUTOR_MEMORY -Xmx$SPARK_EXECUTOR_MEMORY -cp "$SPARK_CLASSPATH" org.apache.spark.executor.CoarseGrainedExecutorBackend --driver-url $SPARK_DRIVER_URL --executor-id $SPARK_EXECUTOR_ID --cores $SPARK_EXECUTOR_CORES --app-id $SPARK_APPLICATION_ID --hostname $SPARK_EXECUTOR_POD_IP diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark-base/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark-base/Dockerfile new file mode 100644 index 0000000000000..8ec1d1baab6b4 --- /dev/null +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark-base/Dockerfile @@ -0,0 +1,43 @@ +# +# 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. +# + +FROM openjdk:8-alpine + +# If this docker file is being used in the context of building your images from a Spark distribution, the docker build +# command should be invoked from the top level directory of the Spark distribution. E.g.: +# docker build -t spark-base:latest -f dockerfiles/spark-base/Dockerfile . + +RUN apk upgrade --no-cache && \ + apk add --no-cache bash tini && \ + mkdir -p /opt/spark && \ + mkdir -p /opt/spark/work-dir \ + touch /opt/spark/RELEASE && \ + rm /bin/sh && \ + ln -sv /bin/bash /bin/sh && \ + chgrp root /etc/passwd && chmod ug+rw /etc/passwd + +COPY jars /opt/spark/jars +COPY bin /opt/spark/bin +COPY sbin /opt/spark/sbin +COPY conf /opt/spark/conf +COPY dockerfiles/spark-base/entrypoint.sh /opt/ + +ENV SPARK_HOME /opt/spark + +WORKDIR /opt/spark/work-dir + +ENTRYPOINT [ "/opt/entrypoint.sh" ] \ No newline at end of file diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark-base/entrypoint.sh b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark-base/entrypoint.sh new file mode 100644 index 0000000000000..d07a2069b1cd1 --- /dev/null +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark-base/entrypoint.sh @@ -0,0 +1,37 @@ +#!/bin/bash +# +# 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. +# + +# echo commands to the terminal output +set -x + +# Check whether there is a passwd entry for the container UID +myuid=$(id -u) +mygid=$(id -g) +uidentry=$(getent passwd $myuid) + +# If there is no passwd entry for the container UID, attempt to create one +if [ -z "$uidentry" ] ; then + if [ -w /etc/passwd ] ; then + echo "$myuid:x:$myuid:$mygid:anonymous uid:$SPARK_HOME:/bin/false" >> /etc/passwd + else + echo "Container ENTRYPOINT failed to add passwd entry for anonymous UID" + fi +fi + +# Execute the container CMD under tini for better hygiene +/sbin/tini -s -- "$@" \ No newline at end of file From 27c67ff2693a1aaac3d0863410c67691a0784ea9 Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Mon, 27 Nov 2017 13:55:38 -0800 Subject: [PATCH 21/37] Addressed first round of review comments --- .../org/apache/spark/deploy/SparkSubmit.scala | 7 +++-- .../org/apache/spark/deploy/k8s/Config.scala | 29 +++++++++---------- .../spark/deploy/k8s/submit/Client.scala | 22 +++++++++++++- ...DriverConfigurationStepsOrchestrator.scala | 8 +++-- .../k8s/submit/LoggingPodStatusWatcher.scala | 10 +++---- .../steps/BaseDriverConfigurationStep.scala | 9 +++--- .../DriverKubernetesCredentialsStep.scala | 4 +-- .../steps/DriverServiceBootstrapStep.scala | 6 ++-- 8 files changed, 59 insertions(+), 36 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index b094751fe721a..1072d53bb11fc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -718,12 +718,13 @@ object SparkSubmit extends CommandLineUtils with Logging { if (isKubernetesCluster) { childMainClass = "org.apache.spark.deploy.k8s.submit.Client" - childArgs ++= Array("--primary-java-resource", args.primaryResource) + if (args.primaryResource != SparkLauncher.NO_RESOURCE) { + childArgs ++= Array("--primary-java-resource", args.primaryResource) + } childArgs ++= Array("--main-class", args.mainClass) if (args.childArgs != null) { args.childArgs.foreach { arg => - childArgs += "--arg" - childArgs += arg + childArgs += ("--arg", arg) } } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index 30068e8893070..a3edbf96b2029 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -64,10 +64,10 @@ private[spark] object Config extends Logging { val KUBERNETES_SERVICE_ACCOUNT_NAME = ConfigBuilder(s"$KUBERNETES_AUTH_DRIVER_CONF_PREFIX.serviceAccountName") - .doc("Service account that is used when running the driver pod. The driver pod uses" + - " this service account when requesting executor pods from the API server. If specific" + - " credentials are given for the driver pod to use, the driver will favor" + - " using those credentials instead.") + .doc("Service account that is used when running the driver pod. The driver pod uses " + + "this service account when requesting executor pods from the API server. If specific " + + "credentials are given for the driver pod to use, the driver will favor " + + "using those credentials instead.") .stringConf .createOptional @@ -85,10 +85,10 @@ private[spark] object Config extends Logging { val KUBERNETES_DRIVER_MEMORY_OVERHEAD = ConfigBuilder("spark.kubernetes.driver.memoryOverhead") - .doc("The amount of off-heap memory (in megabytes) to be allocated for the driver and the" + - " driver submission server. This is memory that accounts for things like VM overheads," + - " interned strings, other native overheads, etc. This tends to grow with the driver's" + - " memory size (typically 6-10%).") + .doc("The amount of off-heap memory (in megabytes) to be allocated for the driver and the " + + "driver submission server. This is memory that accounts for things like VM overheads, " + + "interned strings, other native overheads, etc. This tends to grow with the driver's " + + "memory size (typically 6-10%).") .bytesConf(ByteUnit.MiB) .createOptional @@ -141,8 +141,8 @@ private[spark] object Config extends Logging { val WAIT_FOR_APP_COMPLETION = ConfigBuilder("spark.kubernetes.submission.waitAppCompletion") - .doc("In cluster mode, whether to wait for the application to finish before exiting the" + - " launcher process.") + .doc("In cluster mode, whether to wait for the application to finish before exiting the " + + "launcher process.") .booleanConf .createWithDefault(true) @@ -166,17 +166,16 @@ private[spark] object Config extends Logging { val KUBERNETES_DRIVER_ENV_KEY = "spark.kubernetes.driverEnv." def getK8sMasterUrl(rawMasterString: String): String = { - if (!rawMasterString.startsWith("k8s://")) { - throw new IllegalArgumentException("Master URL should start with k8s:// in Kubernetes mode.") - } + require(rawMasterString.startsWith("k8s://"), + "Master URL should start with k8s:// in Kubernetes mode.") val masterWithoutK8sPrefix = rawMasterString.replaceFirst("k8s://", "") if (masterWithoutK8sPrefix.startsWith("http://") || masterWithoutK8sPrefix.startsWith("https://")) { masterWithoutK8sPrefix } else { val resolvedURL = s"https://$masterWithoutK8sPrefix" - logInfo("No scheme specified for kubernetes master URL, so defaulting to https. Resolved" + - s" URL is $resolvedURL") + logInfo("No scheme specified for kubernetes master URL, so defaulting to https. Resolved " + + s"URL is $resolvedURL") resolvedURL } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala index fe2be3dc6cd68..6a4422956c3d2 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala @@ -33,6 +33,13 @@ import org.apache.spark.deploy.k8s.submit.steps.DriverConfigurationStep import org.apache.spark.internal.Logging import org.apache.spark.util.Utils +/** + * Encapsulates arguments to the submission client. + * + * @param mainAppResource the main application resource + * @param mainClass the main class of the application to run + * @param driverArgs arguments to the driver + */ private[spark] case class ClientArguments( mainAppResource: MainAppResource, mainClass: String, @@ -58,7 +65,7 @@ private[spark] object ClientArguments { } require(mainAppResource.isDefined, - "Main app resource must be defined by either --primary-py-file or --primary-java-resource.") + "Main app resource must be defined by --primary-java-resource.") require(mainClass.isDefined, "Main class must be specified via --main-class") ClientArguments( @@ -68,6 +75,19 @@ private[spark] object ClientArguments { } } +/** + * Submits a Spark application to run on Kubernetes by creating the driver pod and starting a + * watcher that monitors and logs the application status. Waits for the application to terminate if + * spark.kubernetes.submission.waitAppCompletion is true. + * + * @param submissionSteps steps that collectively configure the driver + * @param submissionSparkConf the submission client Spark configuration + * @param kubernetesClient the client to talk to the Kubernetes API server + * @param waitForAppCompletion a flag indicating whether the client should wait for the application + * to complete + * @param appName the application name + * @param loggingPodStatusWatcher a watcher that monitors and logs the application status + */ private[spark] class Client( submissionSteps: Seq[DriverConfigurationStep], submissionSparkConf: SparkConf, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala index 36d428f3574bf..86a445a02a89f 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala @@ -51,8 +51,12 @@ private[spark] class DriverConfigurationStepsOrchestrator( submissionSparkConf, KUBERNETES_DRIVER_LABEL_PREFIX) require(!driverCustomLabels.contains(SPARK_APP_ID_LABEL), "Label with key " + - s" $SPARK_APP_ID_LABEL is not allowed as it is reserved for Spark bookkeeping" + - " operations.") + s"$SPARK_APP_ID_LABEL is not allowed as it is reserved for Spark bookkeeping " + + "operations.") + require(!driverCustomLabels.contains(SPARK_ROLE_LABEL), "Label with key " + + s"$SPARK_ROLE_LABEL is not allowed as it is reserved for Spark bookkeeping " + + "operations.") + val allDriverLabels = driverCustomLabels ++ Map( SPARK_APP_ID_LABEL -> kubernetesAppId, SPARK_ROLE_LABEL -> SPARK_POD_DRIVER_ROLE) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala index be8a32af00afe..60c7447843e3f 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala @@ -41,8 +41,9 @@ private[k8s] trait LoggingPodStatusWatcher extends Watcher[Pod] { * number. */ private[k8s] class LoggingPodStatusWatcherImpl( - appId: String, maybeLoggingInterval: Option[Long]) - extends LoggingPodStatusWatcher with Logging { + appId: String, + maybeLoggingInterval: Option[Long]) + extends LoggingPodStatusWatcher with Logging { private val podCompletedFuture = new CountDownLatch(1) // start timer for periodic logging @@ -66,10 +67,7 @@ private[k8s] class LoggingPodStatusWatcherImpl( override def eventReceived(action: Action, pod: Pod): Unit = { this.pod = Option(pod) action match { - case Action.DELETED => - closeWatch() - - case Action.ERROR => + case Action.DELETED | Action.ERROR => closeWatch() case _ => diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStep.scala index 398ddbeffc4dc..5112b410aa763 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStep.scala @@ -83,10 +83,11 @@ private[spark] class BaseDriverConfigurationStep( " Spark bookkeeping operations.") val driverCustomEnvs = submissionSparkConf.getAllWithPrefix(KUBERNETES_DRIVER_ENV_KEY).toSeq - .map(env => new EnvVarBuilder() - .withName(env._1) - .withValue(env._2) - .build()) + .map(env => + new EnvVarBuilder() + .withName(env._1) + .withValue(env._2) + .build()) val allDriverAnnotations = driverCustomAnnotations ++ Map(SPARK_APP_NAME_ANNOTATION -> appName) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverKubernetesCredentialsStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverKubernetesCredentialsStep.scala index 56856358dfe76..73fd91ff9df15 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverKubernetesCredentialsStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverKubernetesCredentialsStep.scala @@ -237,8 +237,8 @@ private[spark] class DriverKubernetesCredentialsStep( private class OptionSettableSparkConf(sparkConf: SparkConf) { def setOption(configEntry: String, option: Option[String]): SparkConf = { - option.map( opt => { + option.map( opt => sparkConf.set(configEntry, opt) - }).getOrElse(sparkConf) + ).getOrElse(sparkConf) } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverServiceBootstrapStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverServiceBootstrapStep.scala index 5eb98a9ac9e5a..4991735758df7 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverServiceBootstrapStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverServiceBootstrapStep.scala @@ -52,9 +52,9 @@ private[spark] class DriverServiceBootstrapStep( } else { val randomServiceId = clock.getTimeMillis() val shorterServiceName = s"spark-$randomServiceId$DRIVER_SVC_POSTFIX" - logWarning(s"Driver's hostname would preferably be $preferredServiceName, but this is" + - s" too long (must be <= 63 characters). Falling back to use $shorterServiceName" + - " as the driver service's name.") + logWarning(s"Driver's hostname would preferably be $preferredServiceName, but this is " + + s"too long (must be <= $MAX_SERVICE_NAME_LENGTH characters). Falling back to use " + + s"$shorterServiceName as the driver service's name.") shorterServiceName } From 6d597d0b733ba0944486b560b93010b90ab3b86a Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Mon, 27 Nov 2017 21:16:48 -0800 Subject: [PATCH 22/37] Made Client implement the SparkApplication trait --- .../spark/deploy/k8s/submit/Client.scala | 25 ++++++++----------- 1 file changed, 11 insertions(+), 14 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala index 6a4422956c3d2..edfed28b24562 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala @@ -26,6 +26,7 @@ import io.fabric8.kubernetes.api.model._ import io.fabric8.kubernetes.client.KubernetesClient import org.apache.spark.SparkConf +import org.apache.spark.deploy.SparkApplication import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.KubernetesClientFactory @@ -182,9 +183,17 @@ private[spark] class Client( } } -private[spark] object Client { +/** + * Main class and entry point of application submission in KUBERNETES mode. + */ +private[spark] object Client extends SparkApplication { + + override def start(args: Array[String], conf: SparkConf): Unit = { + val parsedArguments = ClientArguments.fromCommandLineArgs(args) + run(parsedArguments, conf) + } - def run(sparkConf: SparkConf, clientArguments: ClientArguments): Unit = { + private def run(clientArguments: ClientArguments, sparkConf: SparkConf): Unit = { val namespace = sparkConf.get(KUBERNETES_NAMESPACE) val kubernetesAppId = s"spark-${UUID.randomUUID().toString.replaceAll("-", "")}" val launchTime = System.currentTimeMillis() @@ -224,16 +233,4 @@ private[spark] object Client { client.run() } } - - /** - * Entry point from SparkSubmit in spark-core - * - * @param args Array of strings that have interchanging values that will be - * parsed by ClientArguments with the identifiers that precede the values - */ - def main(args: Array[String]): Unit = { - val parsedArguments = ClientArguments.fromCommandLineArgs(args) - val sparkConf = new SparkConf() - run(sparkConf, parsedArguments) - } } From 5b9fa39ffbdb2590ca48071f55d9b1abd270268b Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Tue, 28 Nov 2017 08:47:13 -0800 Subject: [PATCH 23/37] Addressed the second round of comments --- .../spark/deploy/SparkSubmitArguments.scala | 12 ++++++++---- .../org/apache/spark/deploy/k8s/Config.scala | 2 +- .../spark/deploy/k8s/submit/Client.scala | 18 ++++++++---------- .../steps/DriverServiceBootstrapStep.scala | 8 ++++---- .../src/main/dockerfiles/driver/Dockerfile | 2 +- .../src/main/dockerfiles/spark-base/Dockerfile | 2 +- .../main/dockerfiles/spark-base/entrypoint.sh | 2 +- 7 files changed, 24 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index a00258e94d9d6..c8b744efdfbe2 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -301,7 +301,9 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S } private def validateKillArguments(): Unit = { - if (!master.startsWith("spark://") && !master.startsWith("mesos://")) { + if (!master.startsWith("spark://") + && !master.startsWith("mesos://") + && !master.startsWith("k8s://")) { SparkSubmit.printErrorAndExit( "Killing submissions is only supported in standalone or Mesos mode!") } @@ -311,7 +313,9 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S } private def validateStatusRequestArguments(): Unit = { - if (!master.startsWith("spark://") && !master.startsWith("mesos://")) { + if (!master.startsWith("spark://") + && !master.startsWith("mesos://") + && !master.startsWith("k8s://")) { SparkSubmit.printErrorAndExit( "Requesting submission statuses is only supported in standalone or Mesos mode!") } @@ -525,8 +529,8 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S outStream.println( s""" |Options: - | --master MASTER_URL spark://host:port, mesos://host:port, yarn, or local - | (Default: local[*]). + | --master MASTER_URL spark://host:port, mesos://host:port, yarn, + | k8s://https://host:port, or local (Default: local[*]). | --deploy-mode DEPLOY_MODE Whether to launch the driver program locally ("client") or | on one of the worker machines inside the cluster ("cluster") | (Default: client). diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index a3edbf96b2029..b07150b35011d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -168,7 +168,7 @@ private[spark] object Config extends Logging { def getK8sMasterUrl(rawMasterString: String): String = { require(rawMasterString.startsWith("k8s://"), "Master URL should start with k8s:// in Kubernetes mode.") - val masterWithoutK8sPrefix = rawMasterString.replaceFirst("k8s://", "") + val masterWithoutK8sPrefix = rawMasterString.substring("k8s://".length) if (masterWithoutK8sPrefix.startsWith("http://") || masterWithoutK8sPrefix.startsWith("https://")) { masterWithoutK8sPrefix diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala index edfed28b24562..ed7c062fbbe93 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala @@ -51,9 +51,9 @@ private[spark] object ClientArguments { def fromCommandLineArgs(args: Array[String]): ClientArguments = { var mainAppResource: Option[MainAppResource] = None var mainClass: Option[String] = None - val driverArgs = mutable.Buffer.empty[String] + val driverArgs = mutable.ArrayBuffer.empty[String] - args.sliding(2, 2).toList.collect { + args.sliding(2, 2).toList.foreach { case Array("--primary-java-resource", primaryJavaResource: String) => mainAppResource = Some(JavaMainAppResource(primaryJavaResource)) case Array("--main-class", clazz: String) => @@ -65,8 +65,6 @@ private[spark] object ClientArguments { throw new RuntimeException(s"Unknown arguments: $invalid") } - require(mainAppResource.isDefined, - "Main app resource must be defined by --primary-java-resource.") require(mainClass.isDefined, "Main class must be specified via --main-class") ClientArguments( @@ -123,10 +121,11 @@ private[spark] class Client( case (confKey, confValue) => s"-D$confKey=$confValue" } ++ driverJavaOptions.map(Utils.splitCommandString).getOrElse(Seq.empty) val driverJavaOptsEnvs: Seq[EnvVar] = resolvedDriverJavaOpts.zipWithIndex.map { - case (option, index) => new EnvVarBuilder() - .withName(s"$ENV_JAVA_OPT_PREFIX$index") - .withValue(option) - .build() + case (option, index) => + new EnvVarBuilder() + .withName(s"$ENV_JAVA_OPT_PREFIX$index") + .withValue(option) + .build() } val resolvedDriverContainer = new ContainerBuilder(currentDriverSpec.driverContainer) @@ -221,8 +220,7 @@ private[spark] object Client extends SparkApplication { KUBERNETES_AUTH_SUBMISSION_CONF_PREFIX, sparkConf, None, - None)) { - kubernetesClient => + None)) { kubernetesClient => val client = new Client( configurationStepsOrchestrator.getAllConfigurationSteps(), sparkConf, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverServiceBootstrapStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverServiceBootstrapStep.scala index 4991735758df7..ee0e26c57d5fe 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverServiceBootstrapStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverServiceBootstrapStep.scala @@ -40,11 +40,11 @@ private[spark] class DriverServiceBootstrapStep( override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { require(submissionSparkConf.getOption(DRIVER_BIND_ADDRESS_KEY).isEmpty, - s"$DRIVER_BIND_ADDRESS_KEY is not supported in Kubernetes mode, as the driver's bind" + - " address is managed and set to the driver pod's IP address.") + s"$DRIVER_BIND_ADDRESS_KEY is not supported in Kubernetes mode, as the driver's bind " + + "address is managed and set to the driver pod's IP address.") require(submissionSparkConf.getOption(DRIVER_HOST_KEY).isEmpty, - s"$DRIVER_HOST_KEY is not supported in Kubernetes mode, as the driver's hostname will be" + - " managed via a Kubernetes service.") + s"$DRIVER_HOST_KEY is not supported in Kubernetes mode, as the driver's hostname will be " + + "managed via a Kubernetes service.") val preferredServiceName = s"$kubernetesResourceNamePrefix$DRIVER_SVC_POSTFIX" val resolvedServiceName = if (preferredServiceName.length <= MAX_SERVICE_NAME_LENGTH) { diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/driver/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/driver/Dockerfile index 6b9ec4d193441..b5f4fb7351978 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/driver/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/driver/Dockerfile @@ -28,4 +28,4 @@ CMD SPARK_CLASSPATH="${SPARK_HOME}/jars/*" && \ readarray -t SPARK_DRIVER_JAVA_OPTS < /tmp/java_opts.txt && \ if ! [ -z ${SPARK_MOUNTED_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_MOUNTED_CLASSPATH:$SPARK_CLASSPATH"; fi && \ if ! [ -z ${SPARK_SUBMIT_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_SUBMIT_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \ - ${JAVA_HOME}/bin/java "${SPARK_DRIVER_JAVA_OPTS[@]}" -cp "$SPARK_CLASSPATH" -Xms$SPARK_DRIVER_MEMORY -Xmx$SPARK_DRIVER_MEMORY -Dspark.driver.bindAddress=$SPARK_DRIVER_BIND_ADDRESS $SPARK_DRIVER_CLASS $SPARK_DRIVER_ARGS \ No newline at end of file + ${JAVA_HOME}/bin/java "${SPARK_DRIVER_JAVA_OPTS[@]}" -cp "$SPARK_CLASSPATH" -Xms$SPARK_DRIVER_MEMORY -Xmx$SPARK_DRIVER_MEMORY -Dspark.driver.bindAddress=$SPARK_DRIVER_BIND_ADDRESS $SPARK_DRIVER_CLASS $SPARK_DRIVER_ARGS diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark-base/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark-base/Dockerfile index 8ec1d1baab6b4..713f61ebb2454 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark-base/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark-base/Dockerfile @@ -40,4 +40,4 @@ ENV SPARK_HOME /opt/spark WORKDIR /opt/spark/work-dir -ENTRYPOINT [ "/opt/entrypoint.sh" ] \ No newline at end of file +ENTRYPOINT [ "/opt/entrypoint.sh" ] diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark-base/entrypoint.sh b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark-base/entrypoint.sh index d07a2069b1cd1..10690ec8de623 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark-base/entrypoint.sh +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark-base/entrypoint.sh @@ -34,4 +34,4 @@ if [ -z "$uidentry" ] ; then fi # Execute the container CMD under tini for better hygiene -/sbin/tini -s -- "$@" \ No newline at end of file +/sbin/tini -s -- "$@" From 5ccadb5b29de157219a388fef28bcbd44e3ec788 Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Thu, 30 Nov 2017 10:16:00 -0800 Subject: [PATCH 24/37] Added missing step for supporting local:// dependencies and addressed more comments --- .../org/apache/spark/deploy/k8s/Config.scala | 16 +++ .../deploy/k8s/KubernetesClientFactory.scala | 104 ------------------ .../spark/deploy/k8s/submit/Client.scala | 10 +- ...DriverConfigurationStepsOrchestrator.scala | 39 ++++++- .../k8s/submit/KubernetesFileUtils.scala | 67 +++++++++++ .../steps/DependencyResolutionStep.scala | 62 +++++++++++ .../DriverKubernetesCredentialsStep.scala | 6 +- ...rConfigurationStepsOrchestratorSuite.scala | 27 ++++- .../steps/DependencyResolutionStepSuite.scala | 81 ++++++++++++++ 9 files changed, 294 insertions(+), 118 deletions(-) delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesClientFactory.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesFileUtils.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DependencyResolutionStep.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DependencyResolutionStepSuite.scala diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index b07150b35011d..085781cd96ca8 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -152,6 +152,22 @@ private[spark] object Config extends Logging { .timeConf(TimeUnit.MILLISECONDS) .createWithDefaultString("1s") + private[spark] val JARS_DOWNLOAD_LOCATION = + ConfigBuilder("spark.kubernetes.mountDependencies.jarsDownloadDir") + .doc("Location to download jars to in the driver and executors. When using" + + " spark-submit, this directory must be empty and will be mounted as an empty directory" + + " volume on the driver and executor pod.") + .stringConf + .createWithDefault("/var/spark-data/spark-jars") + + private[spark] val FILES_DOWNLOAD_LOCATION = + ConfigBuilder("spark.kubernetes.mountDependencies.filesDownloadDir") + .doc("Location to download files to in the driver and executors. When using" + + " spark-submit, this directory must be empty and will be mounted as an empty directory" + + " volume on the driver and executor pods.") + .stringConf + .createWithDefault("/var/spark-data/spark-files") + val KUBERNETES_AUTH_SUBMISSION_CONF_PREFIX = "spark.kubernetes.authenticate.submission" diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesClientFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesClientFactory.scala deleted file mode 100644 index cc3b61efe08d6..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesClientFactory.scala +++ /dev/null @@ -1,104 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.deploy.k8s - -import java.io.File - -import com.google.common.base.Charsets -import com.google.common.io.Files -import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient, KubernetesClient} -import io.fabric8.kubernetes.client.utils.HttpClientUtils -import okhttp3.Dispatcher - -import org.apache.spark.SparkConf -import org.apache.spark.deploy.k8s.Config._ -import org.apache.spark.util.ThreadUtils - -/** - * Spark-opinionated builder for Kubernetes clients. It uses a prefix plus common suffixes to - * parse configuration keys, similar to the manner in which Spark's SecurityManager parses SSL - * options for different components. - */ -private[spark] object KubernetesClientFactory { - - def createKubernetesClient( - master: String, - namespace: Option[String], - kubernetesAuthConfPrefix: String, - sparkConf: SparkConf, - maybeServiceAccountToken: Option[File], - maybeServiceAccountCaCert: Option[File]): KubernetesClient = { - val oauthTokenFileConf = s"$kubernetesAuthConfPrefix.$OAUTH_TOKEN_FILE_CONF_SUFFIX" - val oauthTokenConf = s"$kubernetesAuthConfPrefix.$OAUTH_TOKEN_CONF_SUFFIX" - val oauthTokenFile = sparkConf.getOption(oauthTokenFileConf) - .map(new File(_)) - .orElse(maybeServiceAccountToken) - val oauthTokenValue = sparkConf.getOption(oauthTokenConf) - ConfigurationUtils.requireNandDefined( - oauthTokenFile, - oauthTokenValue, - s"Cannot specify OAuth token through both a file $oauthTokenFileConf and a" + - s" value $oauthTokenConf.") - - val caCertFile = sparkConf - .getOption(s"$kubernetesAuthConfPrefix.$CA_CERT_FILE_CONF_SUFFIX") - .orElse(maybeServiceAccountCaCert.map(_.getAbsolutePath)) - val clientKeyFile = sparkConf - .getOption(s"$kubernetesAuthConfPrefix.$CLIENT_KEY_FILE_CONF_SUFFIX") - val clientCertFile = sparkConf - .getOption(s"$kubernetesAuthConfPrefix.$CLIENT_CERT_FILE_CONF_SUFFIX") - - val config = new ConfigBuilder() - .withApiVersion("v1") - .withMasterUrl(master) - .withWebsocketPingInterval(0) - .withOption(oauthTokenValue) { - (token, configBuilder) => configBuilder.withOauthToken(token) - }.withOption(oauthTokenFile) { - (file, configBuilder) => - configBuilder.withOauthToken(Files.toString(file, Charsets.UTF_8)) - }.withOption(caCertFile) { - (file, configBuilder) => configBuilder.withCaCertFile(file) - }.withOption(clientKeyFile) { - (file, configBuilder) => configBuilder.withClientKeyFile(file) - }.withOption(clientCertFile) { - (file, configBuilder) => configBuilder.withClientCertFile(file) - }.withOption(namespace) { - (ns, configBuilder) => configBuilder.withNamespace(ns) - }.build() - - val baseHttpClient = HttpClientUtils.createHttpClient(config) - val dispatcher = new Dispatcher( - ThreadUtils.newDaemonCachedThreadPool("kubernetes-dispatcher")) - val httpClientWithCustomDispatcher = baseHttpClient.newBuilder() - .dispatcher(dispatcher) - .build() - new DefaultKubernetesClient(httpClientWithCustomDispatcher, config) - } - - private implicit class OptionConfigurableConfigBuilder(val configBuilder: ConfigBuilder) - extends AnyVal { - - def withOption[T] - (option: Option[T]) - (configurator: ((T, ConfigBuilder) => ConfigBuilder)): ConfigBuilder = { - option.map { opt => - configurator(opt, configBuilder) - }.getOrElse(configBuilder) - } - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala index ed7c062fbbe93..0c86cc8bb179b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala @@ -29,7 +29,7 @@ import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkApplication import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.KubernetesClientFactory +import org.apache.spark.deploy.k8s.SparkKubernetesClientFactory import org.apache.spark.deploy.k8s.submit.steps.DriverConfigurationStep import org.apache.spark.internal.Logging import org.apache.spark.util.Utils @@ -37,12 +37,12 @@ import org.apache.spark.util.Utils /** * Encapsulates arguments to the submission client. * - * @param mainAppResource the main application resource + * @param mainAppResource the main application resource if any * @param mainClass the main class of the application to run * @param driverArgs arguments to the driver */ private[spark] case class ClientArguments( - mainAppResource: MainAppResource, + mainAppResource: Option[MainAppResource], mainClass: String, driverArgs: Array[String]) @@ -68,7 +68,7 @@ private[spark] object ClientArguments { require(mainClass.isDefined, "Main class must be specified via --main-class") ClientArguments( - mainAppResource.get, + mainAppResource, mainClass.get, driverArgs.toArray) } @@ -214,7 +214,7 @@ private[spark] object Client extends SparkApplication { clientArguments.driverArgs, sparkConf) - Utils.tryWithResource(KubernetesClientFactory.createKubernetesClient( + Utils.tryWithResource(SparkKubernetesClientFactory.createKubernetesClient( master, Some(namespace), KUBERNETES_AUTH_SUBMISSION_CONF_PREFIX, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala index 86a445a02a89f..27a7afa6b0f55 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala @@ -20,7 +20,8 @@ import org.apache.spark.SparkConf import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.ConfigurationUtils import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.submit.steps.{BaseDriverConfigurationStep, DriverConfigurationStep, DriverKubernetesCredentialsStep, DriverServiceBootstrapStep} +import org.apache.spark.deploy.k8s.submit.steps._ +import org.apache.spark.launcher.SparkLauncher import org.apache.spark.util.SystemClock /** @@ -30,7 +31,7 @@ private[spark] class DriverConfigurationStepsOrchestrator( namespace: String, kubernetesAppId: String, launchTime: Long, - mainAppResource: MainAppResource, + mainAppResource: Option[MainAppResource], appName: String, mainClass: String, appArgs: Array[String], @@ -45,6 +46,8 @@ private[spark] class DriverConfigurationStepsOrchestrator( private val kubernetesResourceNamePrefix = s"$appName-$launchTime".toLowerCase.replaceAll("\\.", "-") private val dockerImagePullPolicy = submissionSparkConf.get(DOCKER_IMAGE_PULL_POLICY) + private val jarsDownloadPath = submissionSparkConf.get(JARS_DOWNLOAD_LOCATION) + private val filesDownloadPath = submissionSparkConf.get(FILES_DOWNLOAD_LOCATION) def getAllConfigurationSteps(): Seq[DriverConfigurationStep] = { val driverCustomLabels = ConfigurationUtils.parsePrefixedKeyValuePairs( @@ -80,9 +83,39 @@ private[spark] class DriverConfigurationStepsOrchestrator( val kubernetesCredentialsStep = new DriverKubernetesCredentialsStep( submissionSparkConf, kubernetesResourceNamePrefix) + val additionalMainAppJar = if (mainAppResource.nonEmpty) { + val mayBeResource = mainAppResource.get match { + case JavaMainAppResource(resource) if resource != SparkLauncher.NO_RESOURCE => + Some(resource) + case _ => Option.empty + } + mayBeResource + } else { + Option.empty + } + + val sparkJars = submissionSparkConf.getOption("spark.jars") + .map(_.split(",")) + .getOrElse(Array.empty[String]) ++ + additionalMainAppJar.toSeq + val sparkFiles = submissionSparkConf.getOption("spark.files") + .map(_.split(",")) + .getOrElse(Array.empty[String]) + + val maybeDependencyResolutionStep = if (sparkJars.nonEmpty || sparkFiles.nonEmpty) { + Some(new DependencyResolutionStep( + sparkJars, + sparkFiles, + jarsDownloadPath, + filesDownloadPath)) + } else { + Option.empty + } + Seq( initialSubmissionStep, driverAddressStep, - kubernetesCredentialsStep) + kubernetesCredentialsStep) ++ + maybeDependencyResolutionStep.toSeq } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesFileUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesFileUtils.scala new file mode 100644 index 0000000000000..a8f9df62c164c --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesFileUtils.scala @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.submit + +import java.io.File + +import org.apache.spark.util.Utils + +private[spark] object KubernetesFileUtils { + + /** + * For the given collection of file URIs, resolves them as follows: + * - File URIs with scheme file:// are resolved to the given download path. + * - File URIs with scheme local:// resolve to just the path of the URI. + * - Otherwise, the URIs are returned as-is. + */ + def resolveSubmittedUris(fileUris: Iterable[String], fileDownloadPath: String) + : Iterable[String] = { + fileUris.map { uri => + val fileUri = Utils.resolveURI(uri) + val fileScheme = Option(fileUri.getScheme).getOrElse("file") + fileScheme match { + case "file" => + val fileName = new File(fileUri.getPath).getName + s"$fileDownloadPath/$fileName" + case "local" => + fileUri.getPath + case _ => uri + } + } + } + + /** + * If any file uri has any scheme other than local:// it is mapped as if the file + * was downloaded to the file download path. Otherwise, it is mapped to the path + * part of the URI. + */ + def resolveFilePaths(fileUris: Iterable[String], fileDownloadPath: String): Iterable[String] = { + fileUris.map { uri => + resolveFilePath(uri, fileDownloadPath) + } + } + + private def resolveFilePath(uri: String, fileDownloadPath: String): String = { + val fileUri = Utils.resolveURI(uri) + if (Option(fileUri.getScheme).getOrElse("file") == "local") { + fileUri.getPath + } else { + val fileName = new File(fileUri.getPath).getName + s"$fileDownloadPath/$fileName" + } + } +} \ No newline at end of file diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DependencyResolutionStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DependencyResolutionStep.scala new file mode 100644 index 0000000000000..36d19ba9dd8df --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DependencyResolutionStep.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.submit.steps + +import java.io.File + +import io.fabric8.kubernetes.api.model.ContainerBuilder + +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.submit.{KubernetesDriverSpec, KubernetesFileUtils} + +/** + * Step that configures the classpath, spark.jars, and spark.files for the driver given that the + * user may provide remote files or files with local:// schemes. + */ +private[spark] class DependencyResolutionStep( + sparkJars: Seq[String], + sparkFiles: Seq[String], + jarsDownloadPath: String, + localFilesDownloadPath: String) extends DriverConfigurationStep { + + override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { + val resolvedSparkJars = KubernetesFileUtils.resolveSubmittedUris(sparkJars, jarsDownloadPath) + val resolvedSparkFiles = KubernetesFileUtils.resolveSubmittedUris( + sparkFiles, localFilesDownloadPath) + val sparkConfResolvedSparkDependencies = driverSpec.driverSparkConf.clone() + if (resolvedSparkJars.nonEmpty) { + sparkConfResolvedSparkDependencies.set("spark.jars", resolvedSparkJars.mkString(",")) + } + if (resolvedSparkFiles.nonEmpty) { + sparkConfResolvedSparkDependencies.set("spark.files", resolvedSparkFiles.mkString(",")) + } + val resolvedClasspath = KubernetesFileUtils.resolveFilePaths(sparkJars, jarsDownloadPath) + val driverContainerWithResolvedClasspath = if (resolvedClasspath.nonEmpty) { + new ContainerBuilder(driverSpec.driverContainer) + .addNewEnv() + .withName(ENV_MOUNTED_CLASSPATH) + .withValue(resolvedClasspath.mkString(File.pathSeparator)) + .endEnv() + .build() + } else { + driverSpec.driverContainer + } + driverSpec.copy( + driverContainer = driverContainerWithResolvedClasspath, + driverSparkConf = sparkConfResolvedSparkDependencies) + } +} \ No newline at end of file diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverKubernetesCredentialsStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverKubernetesCredentialsStep.scala index 73fd91ff9df15..652c9ae3a302c 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverKubernetesCredentialsStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverKubernetesCredentialsStep.scala @@ -213,16 +213,16 @@ private[spark] class DriverKubernetesCredentialsStep( mountedUserSpecified: Option[String], valueMountedFromSubmitter: Option[String], mountedCanonicalLocation: String): Option[String] = { - mountedUserSpecified.orElse(valueMountedFromSubmitter.map( _ => { + mountedUserSpecified.orElse(valueMountedFromSubmitter.map( _ => mountedCanonicalLocation - })) + )) } private def resolveSecretData( mountedUserSpecified: Option[String], valueMountedFromSubmitter: Option[String], secretName: String): Map[String, String] = { - mountedUserSpecified.map { _ => Map.empty[String, String]} + mountedUserSpecified.map { _ => Map.empty[String, String] } .getOrElse { valueMountedFromSubmitter.map { valueBase64 => Map(secretName -> valueBase64) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala index 3a1603fd49b16..c7291d49b465e 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.deploy.k8s.submit import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.k8s.Config.DRIVER_DOCKER_IMAGE -import org.apache.spark.deploy.k8s.submit.steps.{BaseDriverConfigurationStep, DriverConfigurationStep, DriverKubernetesCredentialsStep, DriverServiceBootstrapStep} +import org.apache.spark.deploy.k8s.submit.steps._ class DriverConfigurationStepsOrchestratorSuite extends SparkFunSuite { @@ -30,7 +30,7 @@ class DriverConfigurationStepsOrchestratorSuite extends SparkFunSuite { private val MAIN_CLASS = "org.apache.spark.examples.SparkPi" private val APP_ARGS = Array("arg1", "arg2") - test("Base submission steps without an init-container or python files.") { + test("Base submission steps with a main app resource.") { val sparkConf = new SparkConf(false) .set(DRIVER_DOCKER_IMAGE, DRIVER_IMAGE) val mainAppResource = JavaMainAppResource("local:///var/apps/jars/main.jar") @@ -38,7 +38,28 @@ class DriverConfigurationStepsOrchestratorSuite extends SparkFunSuite { NAMESPACE, APP_ID, LAUNCH_TIME, - mainAppResource, + Some(mainAppResource), + APP_NAME, + MAIN_CLASS, + APP_ARGS, + sparkConf) + validateStepTypes( + orchestrator, + classOf[BaseDriverConfigurationStep], + classOf[DriverServiceBootstrapStep], + classOf[DriverKubernetesCredentialsStep], + classOf[DependencyResolutionStep] + ) + } + + test("Base submission steps without a main app resource.") { + val sparkConf = new SparkConf(false) + .set(DRIVER_DOCKER_IMAGE, DRIVER_IMAGE) + val orchestrator = new DriverConfigurationStepsOrchestrator( + NAMESPACE, + APP_ID, + LAUNCH_TIME, + Option.empty, APP_NAME, MAIN_CLASS, APP_ARGS, diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DependencyResolutionStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DependencyResolutionStepSuite.scala new file mode 100644 index 0000000000000..e00c698be716d --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DependencyResolutionStepSuite.scala @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.submit.steps + +import java.io.File + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model.{ContainerBuilder, HasMetadata, PodBuilder} + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.submit.KubernetesDriverSpec + +private[spark] class DependencyResolutionStepSuite extends SparkFunSuite { + + private val SPARK_JARS = Seq( + "hdfs://localhost:9000/apps/jars/jar1.jar", + "file:///home/user/apps/jars/jar2.jar", + "local:///var/apps/jars/jar3.jar") + + private val SPARK_FILES = Seq( + "file:///home/user/apps/files/file1.txt", + "hdfs://localhost:9000/apps/files/file2.txt", + "local:///var/apps/files/file3.txt") + + private val JARS_DOWNLOAD_PATH = "/mnt/spark-data/jars" + private val FILES_DOWNLOAD_PATH = "/mnt/spark-data/files" + + test("Added dependencies should be resolved in Spark configuration and environment") { + val dependencyResolutionStep = new DependencyResolutionStep( + SPARK_JARS, + SPARK_FILES, + JARS_DOWNLOAD_PATH, + FILES_DOWNLOAD_PATH) + val driverPod = new PodBuilder().build() + val baseDriverSpec = KubernetesDriverSpec( + driverPod = driverPod, + driverContainer = new ContainerBuilder().build(), + driverSparkConf = new SparkConf(false), + otherKubernetesResources = Seq.empty[HasMetadata]) + val preparedDriverSpec = dependencyResolutionStep.configureDriver(baseDriverSpec) + assert(preparedDriverSpec.driverPod === driverPod) + assert(preparedDriverSpec.otherKubernetesResources.isEmpty) + val resolvedSparkJars = preparedDriverSpec.driverSparkConf.get("spark.jars").split(",").toSet + val expectedResolvedSparkJars = Set( + "hdfs://localhost:9000/apps/jars/jar1.jar", + s"$JARS_DOWNLOAD_PATH/jar2.jar", + "/var/apps/jars/jar3.jar") + assert(resolvedSparkJars === expectedResolvedSparkJars) + val resolvedSparkFiles = preparedDriverSpec.driverSparkConf.get("spark.files").split(",").toSet + val expectedResolvedSparkFiles = Set( + s"$FILES_DOWNLOAD_PATH/file1.txt", + s"hdfs://localhost:9000/apps/files/file2.txt", + s"/var/apps/files/file3.txt") + assert(resolvedSparkFiles === expectedResolvedSparkFiles) + val driverEnv = preparedDriverSpec.driverContainer.getEnv.asScala + assert(driverEnv.size === 1) + assert(driverEnv.head.getName === ENV_MOUNTED_CLASSPATH) + val resolvedDriverClasspath = driverEnv.head.getValue.split(File.pathSeparator).toSet + val expectedResolvedDriverClasspath = Set( + s"$JARS_DOWNLOAD_PATH/jar1.jar", + s"$JARS_DOWNLOAD_PATH/jar2.jar", + "/var/apps/jars/jar3.jar") + assert(resolvedDriverClasspath === expectedResolvedDriverClasspath) + } +} \ No newline at end of file From 12f2797296d5aa5a8adf9bbb6cf5d510d747a48f Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Thu, 30 Nov 2017 11:04:24 -0800 Subject: [PATCH 25/37] Fixed Scala style check errors --- .../spark/deploy/k8s/submit/KubernetesFileUtils.scala | 7 ++++--- .../k8s/submit/steps/BaseDriverConfigurationStep.scala | 2 +- .../deploy/k8s/submit/steps/DependencyResolutionStep.scala | 2 +- .../submit/steps/BaseDriverConfigurationStepSuite.scala | 4 ++-- .../k8s/submit/steps/DependencyResolutionStepSuite.scala | 2 +- 5 files changed, 9 insertions(+), 8 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesFileUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesFileUtils.scala index a8f9df62c164c..feba8dad2d94c 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesFileUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesFileUtils.scala @@ -28,8 +28,9 @@ private[spark] object KubernetesFileUtils { * - File URIs with scheme local:// resolve to just the path of the URI. * - Otherwise, the URIs are returned as-is. */ - def resolveSubmittedUris(fileUris: Iterable[String], fileDownloadPath: String) - : Iterable[String] = { + def resolveSubmittedUris( + fileUris: Iterable[String], + fileDownloadPath: String): Iterable[String] = { fileUris.map { uri => val fileUri = Utils.resolveURI(uri) val fileScheme = Option(fileUri.getScheme).getOrElse("file") @@ -64,4 +65,4 @@ private[spark] object KubernetesFileUtils { s"$fileDownloadPath/$fileName" } } -} \ No newline at end of file +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStep.scala index 5112b410aa763..ad50e82100b58 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStep.scala @@ -123,7 +123,7 @@ private[spark] class BaseDriverConfigurationStep( .endEnv() .addNewEnv() .withName(ENV_DRIVER_ARGS) - .withValue(appArgs.mkString(" ")) + .withValue(appArgs.map(arg => "\"" + arg + "\"").mkString(" ")) .endEnv() .addNewEnv() .withName(ENV_DRIVER_BIND_ADDRESS) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DependencyResolutionStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DependencyResolutionStep.scala index 36d19ba9dd8df..96b5d9039ad11 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DependencyResolutionStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DependencyResolutionStep.scala @@ -59,4 +59,4 @@ private[spark] class DependencyResolutionStep( driverContainer = driverContainerWithResolvedClasspath, driverSparkConf = sparkConfResolvedSparkDependencies) } -} \ No newline at end of file +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStepSuite.scala index 8af6f438c2ba0..c34a891d2e3da 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStepSuite.scala @@ -33,7 +33,7 @@ class BaseDriverConfigurationStepSuite extends SparkFunSuite { private val DOCKER_IMAGE_PULL_POLICY = "IfNotPresent" private val APP_NAME = "spark-test" private val MAIN_CLASS = "org.apache.spark.examples.SparkPi" - private val APP_ARGS = Array("arg1", "arg2") + private val APP_ARGS = Array("arg1", "arg2", "arg 3") private val CUSTOM_ANNOTATION_KEY = "customAnnotation" private val CUSTOM_ANNOTATION_VALUE = "customAnnotationValue" private val DRIVER_CUSTOM_ENV_KEY1 = "customDriverEnv1" @@ -82,7 +82,7 @@ class BaseDriverConfigurationStepSuite extends SparkFunSuite { assert(envs(ENV_SUBMIT_EXTRA_CLASSPATH) === "/opt/spark/spark-examples.jar") assert(envs(ENV_DRIVER_MEMORY) === "256M") assert(envs(ENV_DRIVER_MAIN_CLASS) === MAIN_CLASS) - assert(envs(ENV_DRIVER_ARGS) === "arg1 arg2") + assert(envs(ENV_DRIVER_ARGS) === "\"arg1\" \"arg2\" \"arg 3\"") assert(envs(DRIVER_CUSTOM_ENV_KEY1) === "customDriverEnv1") assert(envs(DRIVER_CUSTOM_ENV_KEY2) === "customDriverEnv2") diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DependencyResolutionStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DependencyResolutionStepSuite.scala index e00c698be716d..63460815ec553 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DependencyResolutionStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DependencyResolutionStepSuite.scala @@ -78,4 +78,4 @@ private[spark] class DependencyResolutionStepSuite extends SparkFunSuite { "/var/apps/jars/jar3.jar") assert(resolvedDriverClasspath === expectedResolvedDriverClasspath) } -} \ No newline at end of file +} From c35fe4893e062ac2cf6d738a7d0430b8b98566aa Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Mon, 4 Dec 2017 10:05:50 -0800 Subject: [PATCH 26/37] Addressed another round of comments --- .../org/apache/spark/deploy/SparkSubmit.scala | 10 +++++++--- .../spark/deploy/SparkSubmitArguments.scala | 8 ++------ .../scala/org/apache/spark/util/Utils.scala | 19 +++++++++++++++++++ .../org/apache/spark/util/UtilsSuite.scala | 11 +++++++++++ .../org/apache/spark/deploy/k8s/Config.scala | 15 --------------- .../spark/deploy/k8s/submit/Client.scala | 6 +++--- .../k8s/submit/LoggingPodStatusWatcher.scala | 1 - .../steps/DriverServiceBootstrapStep.scala | 2 +- .../steps/DependencyResolutionStepSuite.scala | 2 +- .../src/main/dockerfiles/driver/Dockerfile | 9 ++++++--- .../src/main/dockerfiles/executor/Dockerfile | 11 +++++++---- .../main/dockerfiles/spark-base/Dockerfile | 10 +++++++--- .../main/dockerfiles/spark-base/entrypoint.sh | 2 +- 13 files changed, 65 insertions(+), 41 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 1072d53bb11fc..81c59fde77554 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -261,7 +261,7 @@ object SparkSubmit extends CommandLineUtils with Logging { case m if m.startsWith("k8s") => KUBERNETES case m if m.startsWith("local") => LOCAL case _ => - printErrorAndExit("Master must either be yarn or start with spark, mesos, local") + printErrorAndExit("Master must either be yarn or start with spark, mesos, k8s, or local") -1 } @@ -296,6 +296,10 @@ object SparkSubmit extends CommandLineUtils with Logging { } } + if (clusterManager == KUBERNETES) { + args.master = Utils.checkAndGetK8sMasterUrl(args.master) + } + // Fail fast, the following modes are not supported or applicable (clusterManager, deployMode) match { case (STANDALONE, CLUSTER) if args.isPython => @@ -304,12 +308,12 @@ object SparkSubmit extends CommandLineUtils with Logging { case (STANDALONE, CLUSTER) if args.isR => printErrorAndExit("Cluster deploy mode is currently not supported for R " + "applications on standalone clusters.") - case (KUBERNETES, CLIENT) => - printErrorAndExit("Client mode is currently not supported for Kubernetes.") case (KUBERNETES, _) if args.isPython => printErrorAndExit("Python applications are currently not supported for Kubernetes.") case (KUBERNETES, _) if args.isR => printErrorAndExit("R applications are currently not supported for Kubernetes.") + case (KUBERNETES, CLIENT) => + printErrorAndExit("Client mode is currently not supported for Kubernetes.") case (LOCAL, CLUSTER) => printErrorAndExit("Cluster deploy mode is not compatible with master \"local\"") case (_, CLUSTER) if isShell(args.primaryResource) => diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index c8b744efdfbe2..1591fd45e0017 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -301,9 +301,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S } private def validateKillArguments(): Unit = { - if (!master.startsWith("spark://") - && !master.startsWith("mesos://") - && !master.startsWith("k8s://")) { + if (!master.startsWith("spark://") && !master.startsWith("mesos://")) { SparkSubmit.printErrorAndExit( "Killing submissions is only supported in standalone or Mesos mode!") } @@ -313,9 +311,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S } private def validateStatusRequestArguments(): Unit = { - if (!master.startsWith("spark://") - && !master.startsWith("mesos://") - && !master.startsWith("k8s://")) { + if (!master.startsWith("spark://") && !master.startsWith("mesos://")) { SparkSubmit.printErrorAndExit( "Requesting submission statuses is only supported in standalone or Mesos mode!") } 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 51bf91614c866..f18703c3e9659 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -2744,6 +2744,25 @@ private[spark] object Utils extends Logging { } } + /** + * Check the validity of the given Kubernetes master URL and return the resolved URL. + */ + def checkAndGetK8sMasterUrl(rawMasterURL: String): String = { + require(rawMasterURL.startsWith("k8s://"), + "Kubernetes master URL must start with k8s://.") + val masterWithoutK8sPrefix = rawMasterURL.substring("k8s://".length) + if (masterWithoutK8sPrefix.startsWith("https://")) { + masterWithoutK8sPrefix + } else if (masterWithoutK8sPrefix.startsWith("http://")) { + logWarning("Kubernetes master URL uses HTTP instead of HTTPS.") + masterWithoutK8sPrefix + } else { + val resolvedURL = s"https://$masterWithoutK8sPrefix" + logInfo("No scheme specified for kubernetes master URL, so defaulting to https. Resolved " + + s"URL is $resolvedURL.") + resolvedURL + } + } } private[util] object CallerContext extends Logging { 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 4d3adeb968e84..d60e52e2c4244 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -1146,6 +1146,17 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { } } + test("check Kubernetes master URL") { + val k8sMasterURLHttps = Utils.checkAndGetK8sMasterUrl("k8s://https://host:port") + assert(k8sMasterURLHttps == "https://host:port") + + val k8sMasterURLHttp = Utils.checkAndGetK8sMasterUrl("k8s://http://host:port") + assert(k8sMasterURLHttp == "http://host:port") + + intercept[IllegalArgumentException] { + Utils.checkAndGetK8sMasterUrl("k8s:https://host:port") + } + } } private class SimpleExtension diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index 085781cd96ca8..fd4f538e7394c 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -180,19 +180,4 @@ private[spark] object Config extends Logging { val KUBERNETES_EXECUTOR_ANNOTATION_PREFIX = "spark.kubernetes.executor.annotation." val KUBERNETES_DRIVER_ENV_KEY = "spark.kubernetes.driverEnv." - - def getK8sMasterUrl(rawMasterString: String): String = { - require(rawMasterString.startsWith("k8s://"), - "Master URL should start with k8s:// in Kubernetes mode.") - val masterWithoutK8sPrefix = rawMasterString.substring("k8s://".length) - if (masterWithoutK8sPrefix.startsWith("http://") - || masterWithoutK8sPrefix.startsWith("https://")) { - masterWithoutK8sPrefix - } else { - val resolvedURL = s"https://$masterWithoutK8sPrefix" - logInfo("No scheme specified for kubernetes master URL, so defaulting to https. Resolved " + - s"URL is $resolvedURL") - resolvedURL - } - } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala index 0c86cc8bb179b..0c11446ae211a 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala @@ -20,7 +20,6 @@ import java.util.{Collections, UUID} import scala.collection.JavaConverters._ import scala.collection.mutable -import scala.util.control.NonFatal import io.fabric8.kubernetes.api.model._ import io.fabric8.kubernetes.client.KubernetesClient @@ -150,7 +149,7 @@ private[spark] class Client( kubernetesClient.resourceList(otherKubernetesResources: _*).createOrReplace() } } catch { - case NonFatal(e) => + case e: Throwable => kubernetesClient.pods().delete(createdDriverPod) throw e } @@ -198,7 +197,8 @@ private[spark] object Client extends SparkApplication { val launchTime = System.currentTimeMillis() val waitForAppCompletion = sparkConf.get(WAIT_FOR_APP_COMPLETION) val appName = sparkConf.getOption("spark.app.name").getOrElse("spark") - val master = getK8sMasterUrl(sparkConf.get("spark.master")) + // The master URL has been checked for validity already in SparkSubmit. + val master = sparkConf.get("spark.master") val loggingInterval = Option(sparkConf.get(REPORT_INTERVAL)).filter(_ => waitForAppCompletion) val loggingPodStatusWatcher = new LoggingPodStatusWatcherImpl( diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala index 60c7447843e3f..fb2fdc2fe9749 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala @@ -101,7 +101,6 @@ private[k8s] class LoggingPodStatusWatcherImpl( } private def formatPodState(pod: Pod): String = { - // TODO include specific container state val details = Seq[(String, String)]( // pod metadata ("pod name", pod.getMetadata.getName), diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverServiceBootstrapStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverServiceBootstrapStep.scala index ee0e26c57d5fe..696d11f15ed95 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverServiceBootstrapStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverServiceBootstrapStep.scala @@ -84,7 +84,7 @@ private[spark] class DriverServiceBootstrapStep( val namespace = submissionSparkConf.get(KUBERNETES_NAMESPACE) val driverHostname = s"${driverService.getMetadata.getName}.$namespace.svc.cluster.local" val resolvedSparkConf = driverSpec.driverSparkConf.clone() - .set(org.apache.spark.internal.config.DRIVER_HOST_ADDRESS, driverHostname) + .set(DRIVER_HOST_KEY, driverHostname) .set("spark.driver.port", driverPort.toString) .set( org.apache.spark.internal.config.DRIVER_BLOCK_MANAGER_PORT, driverBlockManagerPort) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DependencyResolutionStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DependencyResolutionStepSuite.scala index 63460815ec553..991b03cafb76c 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DependencyResolutionStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/DependencyResolutionStepSuite.scala @@ -26,7 +26,7 @@ import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.submit.KubernetesDriverSpec -private[spark] class DependencyResolutionStepSuite extends SparkFunSuite { +class DependencyResolutionStepSuite extends SparkFunSuite { private val SPARK_JARS = Seq( "hdfs://localhost:9000/apps/jars/jar1.jar", diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/driver/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/driver/Dockerfile index b5f4fb7351978..d16349559466d 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/driver/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/driver/Dockerfile @@ -17,9 +17,12 @@ FROM spark-base -# If this docker file is being used in the context of building your images from a Spark distribution, the docker build -# command should be invoked from the top level directory of the Spark distribution. E.g.: -# docker build -t spark-driver:latest -f dockerfiles/driver/Dockerfile . +# Before building the docker image, first build and make a Spark distribution following +# the instructions in http://spark.apache.org/docs/latest/building-spark.html. +# If this docker file is being used in the context of building your images from a Spark +# distribution, the docker build command should be invoked from the top level directory +# of the Spark distribution. E.g.: +# docker build -t spark-driver:latest -f dockerfiles/spark-base/Dockerfile . COPY examples /opt/spark/examples diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/executor/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/executor/Dockerfile index 878120dcbda0b..0e38169b8efdc 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/executor/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/executor/Dockerfile @@ -17,9 +17,12 @@ FROM spark-base -# If this docker file is being used in the context of building your images from a Spark distribution, the docker build -# command should be invoked from the top level directory of the Spark distribution. E.g.: -# docker build -t spark-executor:latest -f dockerfiles/executor/Dockerfile . +# Before building the docker image, first build and make a Spark distribution following +# the instructions in http://spark.apache.org/docs/latest/building-spark.html. +# If this docker file is being used in the context of building your images from a Spark +# distribution, the docker build command should be invoked from the top level directory +# of the Spark distribution. E.g.: +# docker build -t spark-executor:latest -f dockerfiles/spark-base/Dockerfile . COPY examples /opt/spark/examples @@ -28,4 +31,4 @@ CMD SPARK_CLASSPATH="${SPARK_HOME}/jars/*" && \ readarray -t SPARK_EXECUTOR_JAVA_OPTS < /tmp/java_opts.txt && \ if ! [ -z ${SPARK_MOUNTED_CLASSPATH}+x} ]; then SPARK_CLASSPATH="$SPARK_MOUNTED_CLASSPATH:$SPARK_CLASSPATH"; fi && \ if ! [ -z ${SPARK_EXECUTOR_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_EXECUTOR_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \ - ${JAVA_HOME}/bin/java "${SPARK_EXECUTOR_JAVA_OPTS[@]}" -Dspark.executor.port=$SPARK_EXECUTOR_PORT -Xms$SPARK_EXECUTOR_MEMORY -Xmx$SPARK_EXECUTOR_MEMORY -cp "$SPARK_CLASSPATH" org.apache.spark.executor.CoarseGrainedExecutorBackend --driver-url $SPARK_DRIVER_URL --executor-id $SPARK_EXECUTOR_ID --cores $SPARK_EXECUTOR_CORES --app-id $SPARK_APPLICATION_ID --hostname $SPARK_EXECUTOR_POD_IP + ${JAVA_HOME}/bin/java "${SPARK_EXECUTOR_JAVA_OPTS[@]}" -Xms$SPARK_EXECUTOR_MEMORY -Xmx$SPARK_EXECUTOR_MEMORY -cp "$SPARK_CLASSPATH" org.apache.spark.executor.CoarseGrainedExecutorBackend --driver-url $SPARK_DRIVER_URL --executor-id $SPARK_EXECUTOR_ID --cores $SPARK_EXECUTOR_CORES --app-id $SPARK_APPLICATION_ID --hostname $SPARK_EXECUTOR_POD_IP diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark-base/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark-base/Dockerfile index 713f61ebb2454..49e84e195158c 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark-base/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark-base/Dockerfile @@ -17,11 +17,15 @@ FROM openjdk:8-alpine -# If this docker file is being used in the context of building your images from a Spark distribution, the docker build -# command should be invoked from the top level directory of the Spark distribution. E.g.: +# Before building the docker image, first build and make a Spark distribution following +# the instructions in http://spark.apache.org/docs/latest/building-spark.html. +# If this docker file is being used in the context of building your images from a Spark +# distribution, the docker build command should be invoked from the top level directory +# of the Spark distribution. E.g.: # docker build -t spark-base:latest -f dockerfiles/spark-base/Dockerfile . -RUN apk upgrade --no-cache && \ +RUN set -ex && \ + apk upgrade --no-cache && \ apk add --no-cache bash tini && \ mkdir -p /opt/spark && \ mkdir -p /opt/spark/work-dir \ diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark-base/entrypoint.sh b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark-base/entrypoint.sh index 10690ec8de623..82559889f4beb 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark-base/entrypoint.sh +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark-base/entrypoint.sh @@ -17,7 +17,7 @@ # # echo commands to the terminal output -set -x +set -ex # Check whether there is a passwd entry for the container UID myuid=$(id -u) From faa2849b57e158471285df51a9a3b1cb0acb6b68 Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Mon, 4 Dec 2017 11:14:11 -0800 Subject: [PATCH 27/37] Rebased on master and added a constant val for the Client class --- .../main/scala/org/apache/spark/deploy/SparkSubmit.scala | 9 ++++++++- .../scala/org/apache/spark/deploy/SparkSubmitSuite.scala | 2 +- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 81c59fde77554..d581a9bc16f09 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -98,6 +98,7 @@ object SparkSubmit extends CommandLineUtils with Logging { "org.apache.spark.deploy.yarn.YarnClusterApplication" private[deploy] val REST_CLUSTER_SUBMIT_CLASS = classOf[RestSubmissionClientApp].getName() private[deploy] val STANDALONE_CLUSTER_SUBMIT_CLASS = classOf[ClientApp].getName() + private[deploy] val KUBERNETES_CLUSTER_SUBMIT_CLASS = "org.apache.spark.deploy.k8s.submit.Client" // scalastyle:off println private[spark] def printVersionAndExit(): Unit = { @@ -298,6 +299,12 @@ object SparkSubmit extends CommandLineUtils with Logging { if (clusterManager == KUBERNETES) { args.master = Utils.checkAndGetK8sMasterUrl(args.master) + // Make sure YARN is included in our build if we're trying to use it + if (!Utils.classIsLoadable(KUBERNETES_CLUSTER_SUBMIT_CLASS) && !Utils.isTesting) { + printErrorAndExit( + "Could not load KUBERNETES classes. " + + "This copy of Spark may not have been compiled with KUBERNETES support.") + } } // Fail fast, the following modes are not supported or applicable @@ -721,7 +728,7 @@ object SparkSubmit extends CommandLineUtils with Logging { } if (isKubernetesCluster) { - childMainClass = "org.apache.spark.deploy.k8s.submit.Client" + childMainClass = KUBERNETES_CLUSTER_SUBMIT_CLASS if (args.primaryResource != SparkLauncher.NO_RESOURCE) { childArgs ++= Array("--primary-java-resource", args.primaryResource) } diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 15eff93a10776..3c538718f70af 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -406,7 +406,7 @@ class SparkSubmitSuite childArgsMap.get("--primary-java-resource") should be (Some("file:/home/thejar.jar")) childArgsMap.get("--main-class") should be (Some("org.SomeClass")) childArgsMap.get("--arg") should be (Some("arg1")) - mainClass should be ("org.apache.spark.deploy.k8s.submit.Client") + mainClass should be (KUBERNETES_CLUSTER_SUBMIT_CLASS) classpath should have length (0) conf.get("spark.executor.memory") should be ("5g") conf.get("spark.driver.memory") should be ("4g") From 347ed697e3f17546af83a8cd8a9578527cf65384 Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Mon, 4 Dec 2017 16:42:39 -0800 Subject: [PATCH 28/37] Addressed another major round of comments --- .../org/apache/spark/deploy/SparkSubmit.scala | 4 - .../spark/deploy/SparkSubmitArguments.scala | 10 --- .../spark/internal/config/package.scala | 4 + .../spark/deploy/SparkSubmitSuite.scala | 4 +- docs/configuration.md | 9 ++ docs/running-on-yarn.md | 9 +- .../org/apache/spark/deploy/k8s/Config.scala | 12 +-- .../spark/deploy/k8s/submit/Client.scala | 2 +- ...DriverConfigurationStepsOrchestrator.scala | 15 +++- .../k8s/submit/KubernetesFileUtils.scala | 36 ++++---- .../k8s/submit/LoggingPodStatusWatcher.scala | 1 - .../steps/BaseDriverConfigurationStep.scala | 19 ++-- .../steps/DependencyResolutionStep.scala | 4 +- .../DriverKubernetesCredentialsStep.scala | 87 ++++++++++--------- .../BaseDriverConfigurationStepSuite.scala | 2 +- .../org/apache/spark/deploy/yarn/config.scala | 4 - 16 files changed, 105 insertions(+), 117 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index d581a9bc16f09..fe28a07a2c526 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -576,10 +576,6 @@ object SparkSubmit extends CommandLineUtils with Logging { OptionAssigner(args.principal, YARN, ALL_DEPLOY_MODES, confKey = "spark.yarn.principal"), OptionAssigner(args.keytab, YARN, ALL_DEPLOY_MODES, confKey = "spark.yarn.keytab"), - // Kubernetes only - OptionAssigner(args.kubernetesNamespace, KUBERNETES, ALL_DEPLOY_MODES, - confKey = "spark.kubernetes.namespace"), - // Other options OptionAssigner(args.executorCores, STANDALONE | YARN | KUBERNETES, ALL_DEPLOY_MODES, confKey = "spark.executor.cores"), diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 1591fd45e0017..8b8088c80dedc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -81,9 +81,6 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S var submissionToRequestStatusFor: String = null var useRest: Boolean = true // used internally - // Kubernetes only - var kubernetesNamespace: String = null - /** Default properties present in the currently defined defaults file. */ lazy val defaultSparkProperties: HashMap[String, String] = { val defaultProperties = new HashMap[String, String]() @@ -202,10 +199,6 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S keytab = Option(keytab).orElse(sparkProperties.get("spark.yarn.keytab")).orNull principal = Option(principal).orElse(sparkProperties.get("spark.yarn.principal")).orNull - kubernetesNamespace = Option(kubernetesNamespace) - .orElse(sparkProperties.get("spark.kubernetes.namespace")) - .orNull - // Try to set main class from JAR if no --class argument is given if (mainClass == null && !isPython && !isR && primaryResource != null) { val uri = new URI(primaryResource) @@ -461,9 +454,6 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S case KEYTAB => keytab = value - case KUBERNETES_NAMESPACE => - kubernetesNamespace = value - case HELP => printUsageAndExit(0) 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 8fa25c0281493..7097b2ac2aae2 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 @@ -41,6 +41,10 @@ package object config { .bytesConf(ByteUnit.MiB) .createWithDefaultString("1g") + private[spark] val DRIVER_MEMORY_OVERHEAD = ConfigBuilder("spark.driver.memoryOverhead") + .bytesConf(ByteUnit.MiB) + .createOptional + private[spark] val EVENT_LOG_COMPRESS = ConfigBuilder("spark.eventLog.compress") .booleanConf diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 3c538718f70af..a54cd809467ee 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -394,8 +394,8 @@ class SparkSubmitSuite "--master", "k8s://host:port", "--executor-memory", "5g", "--class", "org.SomeClass", - "--kubernetes-namespace", "foo", "--driver-memory", "4g", + "--conf", "spark.kubernetes.namespace=spark", "--conf", "spark.kubernetes.driver.docker.image=bar", "/home/thejar.jar", "arg1") @@ -410,7 +410,7 @@ class SparkSubmitSuite classpath should have length (0) conf.get("spark.executor.memory") should be ("5g") conf.get("spark.driver.memory") should be ("4g") - conf.get("spark.kubernetes.namespace") should be ("foo") + conf.get("spark.kubernetes.namespace") should be ("spark") conf.get("spark.kubernetes.driver.docker.image") should be ("bar") } diff --git a/docs/configuration.md b/docs/configuration.md index ef061dd39dcba..018f8265c93b4 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -157,6 +157,15 @@ of the most common options to set are: or in your default properties file. + + spark.driver.memoryOverhead + driverMemory * 0.10, with minimum of 384 + + The amount of off-heap memory (in megabytes) to be allocated per driver in cluster mode. This is + memory that accounts for things like VM overheads, interned strings, other native overheads, etc. + This tends to grow with the container size (typically 6-10%). + + spark.executor.memory 1g diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 9599d40c545b2..a9ee944b50d41 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -234,18 +234,11 @@ To use a custom metrics.properties for the application master and executors, upd The amount of off-heap memory (in megabytes) to be allocated per executor. This is memory that accounts for things like VM overheads, interned strings, other native overheads, etc. This tends to grow with the executor size (typically 6-10%). - - spark.yarn.driver.memoryOverhead - driverMemory * 0.10, with minimum of 384 - - The amount of off-heap memory (in megabytes) to be allocated per driver in cluster mode. This is memory that accounts for things like VM overheads, interned strings, other native overheads, etc. This tends to grow with the container size (typically 6-10%). - - spark.yarn.am.memoryOverhead AM memory * 0.10, with minimum of 384 - Same as spark.yarn.driver.memoryOverhead, but for the YARN Application Master in client mode. + Same as spark.driver.memoryOverhead, but for the YARN Application Master in client mode. diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index fd4f538e7394c..e98aca4764f30 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -79,19 +79,10 @@ private[spark] object Config extends Logging { val KUBERNETES_EXECUTOR_LIMIT_CORES = ConfigBuilder("spark.kubernetes.executor.limit.cores") - .doc("Specify the hard cpu limit for a single executor pod") + .doc("Specify the hard cpu limit for each executor pod") .stringConf .createOptional - val KUBERNETES_DRIVER_MEMORY_OVERHEAD = - ConfigBuilder("spark.kubernetes.driver.memoryOverhead") - .doc("The amount of off-heap memory (in megabytes) to be allocated for the driver and the " + - "driver submission server. This is memory that accounts for things like VM overheads, " + - "interned strings, other native overheads, etc. This tends to grow with the driver's " + - "memory size (typically 6-10%).") - .bytesConf(ByteUnit.MiB) - .createOptional - // Note that while we set a default for this when we start up the // scheduler, the specific default value is dynamically determined // based on the executor memory. @@ -150,6 +141,7 @@ private[spark] object Config extends Logging { ConfigBuilder("spark.kubernetes.report.interval") .doc("Interval between reports of the current app status in cluster mode.") .timeConf(TimeUnit.MILLISECONDS) + .checkValue(interval => interval > 0, s"Logging interval must be a positive time value.") .createWithDefaultString("1s") private[spark] val JARS_DOWNLOAD_LOCATION = diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala index 0c11446ae211a..f118d31dee44f 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala @@ -199,7 +199,7 @@ private[spark] object Client extends SparkApplication { val appName = sparkConf.getOption("spark.app.name").getOrElse("spark") // The master URL has been checked for validity already in SparkSubmit. val master = sparkConf.get("spark.master") - val loggingInterval = Option(sparkConf.get(REPORT_INTERVAL)).filter(_ => waitForAppCompletion) + val loggingInterval = if (waitForAppCompletion) Some(sparkConf.get(REPORT_INTERVAL)) else None val loggingPodStatusWatcher = new LoggingPodStatusWatcherImpl( kubernetesAppId, loggingInterval) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala index 27a7afa6b0f55..423ff66db3aa6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala @@ -16,6 +16,10 @@ */ package org.apache.spark.deploy.k8s.submit +import java.util.UUID + +import com.google.common.primitives.Longs + import org.apache.spark.SparkConf import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.ConfigurationUtils @@ -43,8 +47,11 @@ private[spark] class DriverConfigurationStepsOrchestrator( // label values are considerably restrictive, e.g. must be no longer than 63 characters in // length. So we generate a separate identifier for the app ID itself, and bookkeeping that // requires finding "all pods for this application" should use the kubernetesAppId. - private val kubernetesResourceNamePrefix = - s"$appName-$launchTime".toLowerCase.replaceAll("\\.", "-") + private val kubernetesResourceNamePrefix = { + val uuid = UUID.nameUUIDFromBytes(Longs.toByteArray(launchTime)) + s"$appName-$uuid".toLowerCase.replaceAll("\\.", "-") + } + private val dockerImagePullPolicy = submissionSparkConf.get(DOCKER_IMAGE_PULL_POLICY) private val jarsDownloadPath = submissionSparkConf.get(JARS_DOWNLOAD_LOCATION) private val filesDownloadPath = submissionSparkConf.get(FILES_DOWNLOAD_LOCATION) @@ -91,7 +98,7 @@ private[spark] class DriverConfigurationStepsOrchestrator( } mayBeResource } else { - Option.empty + None } val sparkJars = submissionSparkConf.getOption("spark.jars") @@ -109,7 +116,7 @@ private[spark] class DriverConfigurationStepsOrchestrator( jarsDownloadPath, filesDownloadPath)) } else { - Option.empty + None } Seq( diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesFileUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesFileUtils.scala index feba8dad2d94c..a38cf55fc3d58 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesFileUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesFileUtils.scala @@ -28,20 +28,11 @@ private[spark] object KubernetesFileUtils { * - File URIs with scheme local:// resolve to just the path of the URI. * - Otherwise, the URIs are returned as-is. */ - def resolveSubmittedUris( + def resolveFileUris( fileUris: Iterable[String], fileDownloadPath: String): Iterable[String] = { fileUris.map { uri => - val fileUri = Utils.resolveURI(uri) - val fileScheme = Option(fileUri.getScheme).getOrElse("file") - fileScheme match { - case "file" => - val fileName = new File(fileUri.getPath).getName - s"$fileDownloadPath/$fileName" - case "local" => - fileUri.getPath - case _ => uri - } + resolveFileUri(uri, fileDownloadPath, false) } } @@ -52,17 +43,26 @@ private[spark] object KubernetesFileUtils { */ def resolveFilePaths(fileUris: Iterable[String], fileDownloadPath: String): Iterable[String] = { fileUris.map { uri => - resolveFilePath(uri, fileDownloadPath) + resolveFileUri(uri, fileDownloadPath, true) } } - private def resolveFilePath(uri: String, fileDownloadPath: String): String = { + private def resolveFileUri( + uri: String, + fileDownloadPath: String, + assumesDownloaded: Boolean): String = { val fileUri = Utils.resolveURI(uri) - if (Option(fileUri.getScheme).getOrElse("file") == "local") { - fileUri.getPath - } else { - val fileName = new File(fileUri.getPath).getName - s"$fileDownloadPath/$fileName" + val fileScheme = Option(fileUri.getScheme).getOrElse("file") + fileScheme match { + case "local" => + fileUri.getPath + case _ => + if (assumesDownloaded || fileScheme == "file") { + val fileName = new File(fileUri.getPath).getName + s"$fileDownloadPath/$fileName" + } else { + uri + } } } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala index fb2fdc2fe9749..173ac541626a7 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala @@ -59,7 +59,6 @@ private[k8s] class LoggingPodStatusWatcherImpl( def start(): Unit = { maybeLoggingInterval.foreach { interval => - require(interval > 0, s"Logging interval must be a positive time value, got: $interval ms.") scheduler.scheduleAtFixedRate(logRunnable, 0, interval, TimeUnit.MILLISECONDS) } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStep.scala index ad50e82100b58..ba2a11b9e6689 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStep.scala @@ -25,6 +25,7 @@ import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.ConfigurationUtils import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.submit.KubernetesDriverSpec +import org.apache.spark.internal.config.{DRIVER_CLASS_PATH, DRIVER_MEMORY, DRIVER_MEMORY_OVERHEAD} /** * Represents the initial setup required for the driver. @@ -43,7 +44,7 @@ private[spark] class BaseDriverConfigurationStep( .getOrElse(s"$kubernetesResourceNamePrefix-driver") private val driverExtraClasspath = submissionSparkConf.get( - org.apache.spark.internal.config.DRIVER_CLASS_PATH) + DRIVER_CLASS_PATH) private val driverDockerImage = submissionSparkConf .get(DRIVER_DOCKER_IMAGE) @@ -55,18 +56,17 @@ private[spark] class BaseDriverConfigurationStep( // Memory settings private val driverMemoryMiB = submissionSparkConf.get( - org.apache.spark.internal.config.DRIVER_MEMORY) + DRIVER_MEMORY) private val driverMemoryString = submissionSparkConf.get( - org.apache.spark.internal.config.DRIVER_MEMORY.key, - org.apache.spark.internal.config.DRIVER_MEMORY.defaultValueString) + DRIVER_MEMORY.key, + DRIVER_MEMORY.defaultValueString) private val memoryOverheadMiB = submissionSparkConf - .get(KUBERNETES_DRIVER_MEMORY_OVERHEAD) + .get(DRIVER_MEMORY_OVERHEAD) .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * driverMemoryMiB).toInt, MEMORY_OVERHEAD_MIN_MIB)) private val driverContainerMemoryWithOverheadMiB = driverMemoryMiB + memoryOverheadMiB - override def configureDriver( - driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { + override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { val driverExtraClasspathEnv = driverExtraClasspath.map { classPath => new EnvVarBuilder() .withName(ENV_SUBMIT_EXTRA_CLASSPATH) @@ -83,11 +83,12 @@ private[spark] class BaseDriverConfigurationStep( " Spark bookkeeping operations.") val driverCustomEnvs = submissionSparkConf.getAllWithPrefix(KUBERNETES_DRIVER_ENV_KEY).toSeq - .map(env => + .map { env => new EnvVarBuilder() .withName(env._1) .withValue(env._2) - .build()) + .build() + } val allDriverAnnotations = driverCustomAnnotations ++ Map(SPARK_APP_NAME_ANNOTATION -> appName) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DependencyResolutionStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DependencyResolutionStep.scala index 96b5d9039ad11..44e0ecffc0e93 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DependencyResolutionStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DependencyResolutionStep.scala @@ -34,8 +34,8 @@ private[spark] class DependencyResolutionStep( localFilesDownloadPath: String) extends DriverConfigurationStep { override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { - val resolvedSparkJars = KubernetesFileUtils.resolveSubmittedUris(sparkJars, jarsDownloadPath) - val resolvedSparkFiles = KubernetesFileUtils.resolveSubmittedUris( + val resolvedSparkJars = KubernetesFileUtils.resolveFileUris(sparkJars, jarsDownloadPath) + val resolvedSparkFiles = KubernetesFileUtils.resolveFileUris( sparkFiles, localFilesDownloadPath) val sparkConfResolvedSparkDependencies = driverSpec.driverSparkConf.clone() if (resolvedSparkJars.nonEmpty) { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverKubernetesCredentialsStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverKubernetesCredentialsStep.scala index 652c9ae3a302c..ccc18908658f1 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverKubernetesCredentialsStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/DriverKubernetesCredentialsStep.scala @@ -39,45 +39,45 @@ private[spark] class DriverKubernetesCredentialsStep( kubernetesResourceNamePrefix: String) extends DriverConfigurationStep { private val maybeMountedOAuthTokenFile = submissionSparkConf.getOption( - s"$KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$OAUTH_TOKEN_FILE_CONF_SUFFIX") + s"$KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$OAUTH_TOKEN_FILE_CONF_SUFFIX") private val maybeMountedClientKeyFile = submissionSparkConf.getOption( - s"$KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX") + s"$KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX") private val maybeMountedClientCertFile = submissionSparkConf.getOption( - s"$KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX") + s"$KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX") private val maybeMountedCaCertFile = submissionSparkConf.getOption( - s"$KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX") + s"$KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX") private val driverServiceAccount = submissionSparkConf.get(KUBERNETES_SERVICE_ACCOUNT_NAME) override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { val driverSparkConf = driverSpec.driverSparkConf.clone() val oauthTokenBase64 = submissionSparkConf - .getOption(s"$KUBERNETES_AUTH_DRIVER_CONF_PREFIX.$OAUTH_TOKEN_CONF_SUFFIX") - .map { token => - BaseEncoding.base64().encode(token.getBytes(StandardCharsets.UTF_8)) - } + .getOption(s"$KUBERNETES_AUTH_DRIVER_CONF_PREFIX.$OAUTH_TOKEN_CONF_SUFFIX") + .map { token => + BaseEncoding.base64().encode(token.getBytes(StandardCharsets.UTF_8)) + } val caCertDataBase64 = safeFileConfToBase64( - s"$KUBERNETES_AUTH_DRIVER_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX", - "Driver CA cert file provided at %s does not exist or is not a file.") + s"$KUBERNETES_AUTH_DRIVER_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX", + "Driver CA cert file") val clientKeyDataBase64 = safeFileConfToBase64( - s"$KUBERNETES_AUTH_DRIVER_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX", - "Driver client key file provided at %s does not exist or is not a file.") + s"$KUBERNETES_AUTH_DRIVER_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX", + "Driver client key file") val clientCertDataBase64 = safeFileConfToBase64( - s"$KUBERNETES_AUTH_DRIVER_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX", - "Driver client cert file provided at %s does not exist or is not a file.") + s"$KUBERNETES_AUTH_DRIVER_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX", + "Driver client cert file") val driverSparkConfWithCredentialsLocations = setDriverPodKubernetesCredentialLocations( - driverSparkConf, - oauthTokenBase64, - caCertDataBase64, - clientKeyDataBase64, - clientCertDataBase64) + driverSparkConf, + oauthTokenBase64, + caCertDataBase64, + clientKeyDataBase64, + clientCertDataBase64) val kubernetesCredentialsSecret = createCredentialsSecret( - oauthTokenBase64, - caCertDataBase64, - clientKeyDataBase64, - clientCertDataBase64) + oauthTokenBase64, + caCertDataBase64, + clientKeyDataBase64, + clientCertDataBase64) val driverPodWithMountedKubernetesCredentials = kubernetesCredentialsSecret.map { secret => new PodBuilder(driverSpec.driverPod) @@ -123,19 +123,15 @@ private[spark] class DriverKubernetesCredentialsStep( driverClientCertDataBase64: Option[String]): Option[Secret] = { val allSecretData = resolveSecretData( - maybeMountedClientKeyFile, driverClientKeyDataBase64, DRIVER_CREDENTIALS_CLIENT_KEY_SECRET_NAME) ++ resolveSecretData( - maybeMountedClientCertFile, driverClientCertDataBase64, DRIVER_CREDENTIALS_CLIENT_CERT_SECRET_NAME) ++ resolveSecretData( - maybeMountedCaCertFile, driverCaCertDataBase64, DRIVER_CREDENTIALS_CA_CERT_SECRET_NAME) ++ resolveSecretData( - maybeMountedOAuthTokenFile, driverOAuthTokenBase64, DRIVER_CREDENTIALS_OAUTH_TOKEN_SECRET_NAME) @@ -198,13 +194,12 @@ private[spark] class DriverKubernetesCredentialsStep( sparkConfWithCredentialLocations } - private def safeFileConfToBase64( - conf: String, - fileNotFoundFormatString: String): Option[String] = { + private def safeFileConfToBase64(conf: String, fileType: String): Option[String] = { submissionSparkConf.getOption(conf) .map(new File(_)) .map { file => - require(file.isFile, String.format(fileNotFoundFormatString, file.getAbsolutePath)) + require(file.isFile, String.format("%s provided at %s does not exist or is not a file.", + fileType, file.getAbsolutePath)) BaseEncoding.base64().encode(Files.toByteArray(file)) } } @@ -213,21 +208,26 @@ private[spark] class DriverKubernetesCredentialsStep( mountedUserSpecified: Option[String], valueMountedFromSubmitter: Option[String], mountedCanonicalLocation: String): Option[String] = { - mountedUserSpecified.orElse(valueMountedFromSubmitter.map( _ => + mountedUserSpecified.orElse(valueMountedFromSubmitter.map { _ => mountedCanonicalLocation - )) + }) } + /** + * Resolve a Kubernetes secret data entry from an optional client credential used by the + * driver to talk to the Kubernetes API server. + * + * @param userSpecifiedCredential the optional user-specified client credential. + * @param secretName name of the Kubernetes secret storing the client credential. + * @return a secret data entry in the form of a map from the secret name to the secret data, + * which may be empty if the user-specified credential is empty. + */ private def resolveSecretData( - mountedUserSpecified: Option[String], - valueMountedFromSubmitter: Option[String], + userSpecifiedCredential: Option[String], secretName: String): Map[String, String] = { - mountedUserSpecified.map { _ => Map.empty[String, String] } - .getOrElse { - valueMountedFromSubmitter.map { valueBase64 => - Map(secretName -> valueBase64) - }.getOrElse(Map.empty[String, String]) - } + userSpecifiedCredential.map { valueBase64 => + Map(secretName -> valueBase64) + }.getOrElse(Map.empty[String, String]) } private implicit def augmentSparkConf(sparkConf: SparkConf): OptionSettableSparkConf = { @@ -237,8 +237,9 @@ private[spark] class DriverKubernetesCredentialsStep( private class OptionSettableSparkConf(sparkConf: SparkConf) { def setOption(configEntry: String, option: Option[String]): SparkConf = { - option.map( opt => + option.foreach { opt => sparkConf.set(configEntry, opt) - ).getOrElse(sparkConf) + } + sparkConf } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStepSuite.scala index c34a891d2e3da..83c5f98254829 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/BaseDriverConfigurationStepSuite.scala @@ -46,7 +46,7 @@ class BaseDriverConfigurationStepSuite extends SparkFunSuite { .set("spark.driver.cores", "2") .set(KUBERNETES_DRIVER_LIMIT_CORES, "4") .set(org.apache.spark.internal.config.DRIVER_MEMORY.key, "256M") - .set(KUBERNETES_DRIVER_MEMORY_OVERHEAD, 200L) + .set(org.apache.spark.internal.config.DRIVER_MEMORY_OVERHEAD, 200L) .set(DRIVER_DOCKER_IMAGE, "spark-driver:latest") .set(s"$KUBERNETES_DRIVER_ANNOTATION_PREFIX$CUSTOM_ANNOTATION_KEY", CUSTOM_ANNOTATION_VALUE) .set(s"$KUBERNETES_DRIVER_ENV_KEY$DRIVER_CUSTOM_ENV_KEY1", "customDriverEnv1") diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala index e1af8ba087d6e..d401280e6bdbc 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala @@ -217,10 +217,6 @@ package object config { .intConf .createWithDefault(1) - private[spark] val DRIVER_MEMORY_OVERHEAD = ConfigBuilder("spark.yarn.driver.memoryOverhead") - .bytesConf(ByteUnit.MiB) - .createOptional - /* Executor configuration. */ private[spark] val EXECUTOR_CORES = ConfigBuilder("spark.executor.cores") From 0e8ca012dc6d5dfd5645f45b6d1bb84e1e9e72a5 Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Tue, 5 Dec 2017 10:42:50 -0800 Subject: [PATCH 29/37] Addressed one more round of comments --- .../scala/org/apache/spark/SparkConf.scala | 4 ++- .../scala/org/apache/spark/util/Utils.scala | 27 ++++++++++++++----- .../spark/deploy/SparkSubmitSuite.scala | 1 + .../org/apache/spark/util/UtilsSuite.scala | 10 +++++++ .../launcher/SparkSubmitOptionParser.java | 6 +---- .../spark/deploy/k8s/submit/Client.scala | 15 +++++++---- ...DriverConfigurationStepsOrchestrator.scala | 11 +++----- 7 files changed, 49 insertions(+), 25 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 0e08ff65e4784..d533f00d6cfc9 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -668,7 +668,9 @@ private[spark] object SparkConf extends Logging { MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM.key -> Seq( AlternateConfig("spark.reducer.maxReqSizeShuffleToMem", "2.3")), LISTENER_BUS_EVENT_QUEUE_CAPACITY.key -> Seq( - AlternateConfig("spark.scheduler.listenerbus.eventqueue.size", "2.3")) + AlternateConfig("spark.scheduler.listenerbus.eventqueue.size", "2.3")), + "spark.driver.memoryOverhead" -> Seq( + AlternateConfig("spark.yarn.driver.memoryOverhead", "2.3")) ) /** 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 f18703c3e9659..a237811f8f608 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -2751,16 +2751,29 @@ private[spark] object Utils extends Logging { require(rawMasterURL.startsWith("k8s://"), "Kubernetes master URL must start with k8s://.") val masterWithoutK8sPrefix = rawMasterURL.substring("k8s://".length) - if (masterWithoutK8sPrefix.startsWith("https://")) { - masterWithoutK8sPrefix - } else if (masterWithoutK8sPrefix.startsWith("http://")) { - logWarning("Kubernetes master URL uses HTTP instead of HTTPS.") - masterWithoutK8sPrefix - } else { + + // To handle master URLs, e.g., k8s://host:port. + if (!masterWithoutK8sPrefix.contains("://")) { val resolvedURL = s"https://$masterWithoutK8sPrefix" logInfo("No scheme specified for kubernetes master URL, so defaulting to https. Resolved " + s"URL is $resolvedURL.") - resolvedURL + return resolvedURL + } + + val masterScheme = new URI(masterWithoutK8sPrefix).getScheme + masterScheme.toLowerCase match { + case "https" => + masterWithoutK8sPrefix + case "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) } } } diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index a54cd809467ee..4c7248d801e7c 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -408,6 +408,7 @@ class SparkSubmitSuite childArgsMap.get("--arg") should be (Some("arg1")) mainClass should be (KUBERNETES_CLUSTER_SUBMIT_CLASS) classpath should have length (0) + conf.get("spark.master") should be ("https://host:port") conf.get("spark.executor.memory") should be ("5g") conf.get("spark.driver.memory") should be ("4g") conf.get("spark.kubernetes.namespace") should be ("spark") 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 d60e52e2c4244..3e71b05f1c086 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -1153,9 +1153,19 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { val k8sMasterURLHttp = Utils.checkAndGetK8sMasterUrl("k8s://http://host:port") assert(k8sMasterURLHttp == "http://host:port") + val k8sMasterURLWithoutScheme = Utils.checkAndGetK8sMasterUrl("k8s://127.0.0.1:8443") + assert(k8sMasterURLWithoutScheme == "https://127.0.0.1:8443") + + val k8sMasterURLWithoutScheme2 = Utils.checkAndGetK8sMasterUrl("k8s://127.0.0.1") + assert(k8sMasterURLWithoutScheme2 == "https://127.0.0.1") + intercept[IllegalArgumentException] { Utils.checkAndGetK8sMasterUrl("k8s:https://host:port") } + + intercept[IllegalArgumentException] { + Utils.checkAndGetK8sMasterUrl("k8s://foo://host:port") + } } } diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java index c6717f127eb73..4ef33ebd31a6a 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java @@ -76,9 +76,6 @@ class SparkSubmitOptionParser { protected final String PRINCIPAL = "--principal"; protected final String QUEUE = "--queue"; - // Kubernetes-only options. - protected final String KUBERNETES_NAMESPACE = "--kubernetes-namespace"; - /** * This is the canonical list of spark-submit options. Each entry in the array contains the * different aliases for the same option; the first element of each entry is the "official" @@ -117,8 +114,7 @@ class SparkSubmitOptionParser { { QUEUE }, { REPOSITORIES }, { STATUS }, - { TOTAL_EXECUTOR_CORES }, - { KUBERNETES_NAMESPACE }, + { TOTAL_EXECUTOR_CORES } }; /** diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala index f118d31dee44f..4e4abd67ffdce 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala @@ -20,6 +20,7 @@ import java.util.{Collections, UUID} import scala.collection.JavaConverters._ import scala.collection.mutable +import scala.util.control.NonFatal import io.fabric8.kubernetes.api.model._ import io.fabric8.kubernetes.client.KubernetesClient @@ -137,10 +138,10 @@ private[spark] class Client( .build() Utils.tryWithResource( - kubernetesClient - .pods() - .withName(resolvedDriverPod.getMetadata.getName) - .watch(loggingPodStatusWatcher)) { _ => + kubernetesClient + .pods() + .withName(resolvedDriverPod.getMetadata.getName) + .watch(loggingPodStatusWatcher)) { _ => val createdDriverPod = kubernetesClient.pods().create(resolvedDriverPod) try { if (currentDriverSpec.otherKubernetesResources.nonEmpty) { @@ -149,7 +150,7 @@ private[spark] class Client( kubernetesClient.resourceList(otherKubernetesResources: _*).createOrReplace() } } catch { - case e: Throwable => + case NonFatal(e) => kubernetesClient.pods().delete(createdDriverPod) throw e } @@ -193,6 +194,10 @@ private[spark] object Client extends SparkApplication { private def run(clientArguments: ClientArguments, sparkConf: SparkConf): Unit = { val namespace = sparkConf.get(KUBERNETES_NAMESPACE) + // For constructing the app ID, we can't use the Spark application name, as the app ID is going + // to be added as a label to group resources belonging to the same application. Label values are + // considerably restrictive, e.g. must be no longer than 63 characters in length. So we generate + // a unique app ID (captured by spark.app.id) in the format below. val kubernetesAppId = s"spark-${UUID.randomUUID().toString.replaceAll("-", "")}" val launchTime = System.currentTimeMillis() val waitForAppCompletion = sparkConf.get(WAIT_FOR_APP_COMPLETION) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala index 423ff66db3aa6..d55f75faf657a 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala @@ -41,14 +41,11 @@ private[spark] class DriverConfigurationStepsOrchestrator( appArgs: Array[String], submissionSparkConf: SparkConf) { - // The resource name prefix is derived from the application name, making it easy to connect the - // names of the Kubernetes resources from e.g. kubectl or the Kubernetes dashboard to the - // application the user submitted. However, we can't use the application name in the label, as - // label values are considerably restrictive, e.g. must be no longer than 63 characters in - // length. So we generate a separate identifier for the app ID itself, and bookkeeping that - // requires finding "all pods for this application" should use the kubernetesAppId. + // The resource name prefix is derived from the Spark application name, making it easy to connect + // the names of the Kubernetes resources from e.g. kubectl or the Kubernetes dashboard to the + // application the user submitted. private val kubernetesResourceNamePrefix = { - val uuid = UUID.nameUUIDFromBytes(Longs.toByteArray(launchTime)) + val uuid = UUID.nameUUIDFromBytes(Longs.toByteArray(launchTime)).toString.replaceAll("-", "") s"$appName-$uuid".toLowerCase.replaceAll("\\.", "-") } From 3a0b8e301e0c47d333a65267ab9e2dbb1a06968f Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Wed, 6 Dec 2017 10:07:12 -0800 Subject: [PATCH 30/37] Removed mentioning of kubernetes-namespace --- .../scala/org/apache/spark/deploy/SparkSubmitArguments.scala | 5 ----- .../src/main/scala/org/apache/spark/deploy/k8s/Config.scala | 4 +--- 2 files changed, 1 insertion(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 8b8088c80dedc..9db7a1fe3106d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -590,11 +590,6 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S | the node running the Application Master via the Secure | Distributed Cache, for renewing the login tickets and the | delegation tokens periodically. - | - | Kubernetes only: - | --kubernetes-namespace NS The namespace in the Kubernetes cluster within which the - | application must be launched. The namespace must already - | exist in the cluster. (Default: default). """.stripMargin ) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index e98aca4764f30..fd5dd7d7ff322 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -26,9 +26,7 @@ private[spark] object Config extends Logging { val KUBERNETES_NAMESPACE = ConfigBuilder("spark.kubernetes.namespace") - .doc("The namespace that will be used for running the driver and executor pods. When using " + - "spark-submit in cluster mode, this can also be passed to spark-submit via the " + - "--kubernetes-namespace command line argument.") + .doc("The namespace that will be used for running the driver and executor pods.") .stringConf .createWithDefault("default") From 83d0b9cc29eec2654308344ac8b2208e5e3f08df Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Thu, 7 Dec 2017 11:09:32 -0800 Subject: [PATCH 31/37] Fixed a couple of bugs found during manual tests --- .../scala/org/apache/spark/deploy/SparkSubmit.scala | 3 ++- core/src/main/scala/org/apache/spark/util/Utils.scala | 10 +++++++--- .../org/apache/spark/deploy/SparkSubmitSuite.scala | 2 +- .../test/scala/org/apache/spark/util/UtilsSuite.scala | 8 ++++---- ...{Client.scala => KubernetesClientApplication.scala} | 5 +++-- 5 files changed, 17 insertions(+), 11 deletions(-) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/{Client.scala => KubernetesClientApplication.scala} (97%) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index fe28a07a2c526..930af3103f26c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -98,7 +98,8 @@ object SparkSubmit extends CommandLineUtils with Logging { "org.apache.spark.deploy.yarn.YarnClusterApplication" private[deploy] val REST_CLUSTER_SUBMIT_CLASS = classOf[RestSubmissionClientApp].getName() private[deploy] val STANDALONE_CLUSTER_SUBMIT_CLASS = classOf[ClientApp].getName() - private[deploy] val KUBERNETES_CLUSTER_SUBMIT_CLASS = "org.apache.spark.deploy.k8s.submit.Client" + private[deploy] val KUBERNETES_CLUSTER_SUBMIT_CLASS = + "org.apache.spark.deploy.k8s.submit.KubernetesClientApplication" // scalastyle:off println private[spark] def printVersionAndExit(): Unit = { 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 a237811f8f608..1ed09dc489440 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -2745,7 +2745,9 @@ private[spark] object Utils extends Logging { } /** - * Check the validity of the given Kubernetes master URL and return the resolved URL. + * Check the validity of the given Kubernetes master URL and return the resolved URL. Prefix + * "k8s:" is appended to the resolved URL as the prefix is used by KubernetesClusterManager + * in canCreate to determine if the KubernetesClusterManager should be used. */ def checkAndGetK8sMasterUrl(rawMasterURL: String): String = { require(rawMasterURL.startsWith("k8s://"), @@ -2757,11 +2759,11 @@ private[spark] object Utils extends Logging { val resolvedURL = s"https://$masterWithoutK8sPrefix" logInfo("No scheme specified for kubernetes master URL, so defaulting to https. Resolved " + s"URL is $resolvedURL.") - return resolvedURL + return s"k8s:$resolvedURL" } val masterScheme = new URI(masterWithoutK8sPrefix).getScheme - masterScheme.toLowerCase match { + val resolvedURL = masterScheme.toLowerCase match { case "https" => masterWithoutK8sPrefix case "http" => @@ -2775,6 +2777,8 @@ private[spark] object Utils extends Logging { case _ => throw new IllegalArgumentException("Invalid Kubernetes master scheme: " + masterScheme) } + + return s"k8s:$resolvedURL" } } diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 4c7248d801e7c..35594ec47c941 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -408,7 +408,7 @@ class SparkSubmitSuite childArgsMap.get("--arg") should be (Some("arg1")) mainClass should be (KUBERNETES_CLUSTER_SUBMIT_CLASS) classpath should have length (0) - conf.get("spark.master") should be ("https://host:port") + conf.get("spark.master") should be ("k8s:https://host:port") conf.get("spark.executor.memory") should be ("5g") conf.get("spark.driver.memory") should be ("4g") conf.get("spark.kubernetes.namespace") should be ("spark") 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 3e71b05f1c086..18c9adccd7c59 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -1148,16 +1148,16 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { test("check Kubernetes master URL") { val k8sMasterURLHttps = Utils.checkAndGetK8sMasterUrl("k8s://https://host:port") - assert(k8sMasterURLHttps == "https://host:port") + assert(k8sMasterURLHttps == "k8s:https://host:port") val k8sMasterURLHttp = Utils.checkAndGetK8sMasterUrl("k8s://http://host:port") - assert(k8sMasterURLHttp == "http://host:port") + assert(k8sMasterURLHttp == "k8s:http://host:port") val k8sMasterURLWithoutScheme = Utils.checkAndGetK8sMasterUrl("k8s://127.0.0.1:8443") - assert(k8sMasterURLWithoutScheme == "https://127.0.0.1:8443") + assert(k8sMasterURLWithoutScheme == "k8s:https://127.0.0.1:8443") val k8sMasterURLWithoutScheme2 = Utils.checkAndGetK8sMasterUrl("k8s://127.0.0.1") - assert(k8sMasterURLWithoutScheme2 == "https://127.0.0.1") + assert(k8sMasterURLWithoutScheme2 == "k8s:https://127.0.0.1") intercept[IllegalArgumentException] { Utils.checkAndGetK8sMasterUrl("k8s:https://host:port") diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala similarity index 97% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala index 4e4abd67ffdce..4d17608c602d8 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala @@ -185,7 +185,7 @@ private[spark] class Client( /** * Main class and entry point of application submission in KUBERNETES mode. */ -private[spark] object Client extends SparkApplication { +private[spark] class KubernetesClientApplication extends SparkApplication { override def start(args: Array[String], conf: SparkConf): Unit = { val parsedArguments = ClientArguments.fromCommandLineArgs(args) @@ -203,7 +203,8 @@ private[spark] object Client extends SparkApplication { val waitForAppCompletion = sparkConf.get(WAIT_FOR_APP_COMPLETION) val appName = sparkConf.getOption("spark.app.name").getOrElse("spark") // The master URL has been checked for validity already in SparkSubmit. - val master = sparkConf.get("spark.master") + // We just need to get rid of the "k8s:" prefix here. + val master = sparkConf.get("spark.master").substring("k8s:".length) val loggingInterval = if (waitForAppCompletion) Some(sparkConf.get(REPORT_INTERVAL)) else None val loggingPodStatusWatcher = new LoggingPodStatusWatcherImpl( From 44c40b1b7d546e4d214aff78c282c768aa0e3d42 Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Thu, 7 Dec 2017 21:42:10 -0800 Subject: [PATCH 32/37] Guard against client mode in SparkContext --- core/src/main/scala/org/apache/spark/SparkContext.scala | 4 ++++ .../src/test/scala/org/apache/spark/SparkContextSuite.scala | 6 ++++++ 2 files changed, 10 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 71f1e7c7321bc..0565011f9fd1e 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -380,6 +380,10 @@ class SparkContext(config: SparkConf) extends Logging { "Deployment to YARN is not supported directly by SparkContext. Please use spark-submit.") } + if (master.startsWith("k8s") && deployMode == "client") { + throw new SparkException("Client mode is currently not supported for Kubernetes.") + } + if (_conf.getBoolean("spark.logConf", false)) { logInfo("Spark configuration:\n" + _conf.toDebugString) } diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 2bde8757dae5d..37fcc93c62fa8 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -550,6 +550,12 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu } } + test("client mode with a k8s master url") { + intercept[SparkException] { + sc = new SparkContext("k8s://https://host:port", "test", new SparkConf()) + } + } + testCancellingTasks("that raise interrupted exception on cancel") { Thread.sleep(9999999) } From 67bc847e4f71bd6974735ac43090a0df266bc5f3 Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Fri, 8 Dec 2017 10:56:48 -0800 Subject: [PATCH 33/37] Added libc6-compat into the base docker image --- .../docker/src/main/dockerfiles/spark-base/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark-base/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark-base/Dockerfile index 49e84e195158c..20316c9c5098a 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark-base/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark-base/Dockerfile @@ -26,7 +26,7 @@ FROM openjdk:8-alpine RUN set -ex && \ apk upgrade --no-cache && \ - apk add --no-cache bash tini && \ + apk add --no-cache bash tini libc6-compat && \ mkdir -p /opt/spark && \ mkdir -p /opt/spark/work-dir \ touch /opt/spark/RELEASE && \ From 7d2b30373b2e4d8d5311e10c3f9a62a2d900d568 Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Fri, 8 Dec 2017 11:39:48 -0800 Subject: [PATCH 34/37] Addressed latest comments --- .../src/main/scala/org/apache/spark/SparkConf.scala | 6 ++++-- .../main/scala/org/apache/spark/SparkContext.scala | 5 ----- .../org/apache/spark/internal/config/package.scala | 4 ++++ docs/configuration.md | 9 +++++++++ docs/running-on-yarn.md | 7 ------- .../spark/launcher/SparkSubmitOptionParser.java | 2 +- .../scala/org/apache/spark/deploy/k8s/Config.scala | 11 ----------- .../DriverConfigurationStepsOrchestrator.scala | 2 +- .../scheduler/cluster/k8s/ExecutorPodFactory.scala | 13 ++++++------- .../cluster/k8s/KubernetesClusterManager.scala | 6 +++++- .../scala/org/apache/spark/deploy/yarn/config.scala | 4 ---- 11 files changed, 30 insertions(+), 39 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index d533f00d6cfc9..4b1286d91e8f3 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -669,8 +669,10 @@ private[spark] object SparkConf extends Logging { AlternateConfig("spark.reducer.maxReqSizeShuffleToMem", "2.3")), LISTENER_BUS_EVENT_QUEUE_CAPACITY.key -> Seq( AlternateConfig("spark.scheduler.listenerbus.eventqueue.size", "2.3")), - "spark.driver.memoryOverhead" -> Seq( - AlternateConfig("spark.yarn.driver.memoryOverhead", "2.3")) + DRIVER_MEMORY_OVERHEAD.key -> Seq( + AlternateConfig("spark.yarn.driver.memoryOverhead", "2.3")), + EXECUTOR_MEMORY_OVERHEAD.key -> Seq( + AlternateConfig("spark.yarn.executor.memoryOverhead", "2.3")) ) /** diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 0565011f9fd1e..92e13ce1ba042 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -18,7 +18,6 @@ package org.apache.spark import java.io._ -import java.lang.reflect.Constructor import java.net.URI import java.util.{Arrays, Locale, Properties, ServiceLoader, UUID} import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap} @@ -380,10 +379,6 @@ class SparkContext(config: SparkConf) extends Logging { "Deployment to YARN is not supported directly by SparkContext. Please use spark-submit.") } - if (master.startsWith("k8s") && deployMode == "client") { - throw new SparkException("Client mode is currently not supported for Kubernetes.") - } - if (_conf.getBoolean("spark.logConf", false)) { logInfo("Spark configuration:\n" + _conf.toDebugString) } 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 7097b2ac2aae2..172ba85359da7 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 @@ -84,6 +84,10 @@ package object config { .bytesConf(ByteUnit.MiB) .createWithDefaultString("1g") + private[spark] val EXECUTOR_MEMORY_OVERHEAD = ConfigBuilder("spark.executor.memoryOverhead") + .bytesConf(ByteUnit.MiB) + .createOptional + private[spark] val MEMORY_OFFHEAP_ENABLED = ConfigBuilder("spark.memory.offHeap.enabled") .doc("If true, Spark will attempt to use off-heap memory for certain operations. " + "If off-heap memory use is enabled, then spark.memory.offHeap.size must be positive.") diff --git a/docs/configuration.md b/docs/configuration.md index 018f8265c93b4..3d72d0dd9fabb 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -173,6 +173,15 @@ of the most common options to set are: Amount of memory to use per executor process (e.g. 2g, 8g). + + spark.executor.memoryOverhead + executorMemory * 0.10, with minimum of 384 + + The amount of off-heap memory (in megabytes) to be allocated per executor. This is memory that + accounts for things like VM overheads, interned strings, other native overheads, etc. This tends + to grow with the executor size (typically 6-10%). + + spark.extraListeners (none) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index a9ee944b50d41..7e2386f33b583 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -227,13 +227,6 @@ To use a custom metrics.properties for the application master and executors, upd The number of executors for static allocation. With spark.dynamicAllocation.enabled, the initial set of executors will be at least this large. - - spark.yarn.executor.memoryOverhead - executorMemory * 0.10, with minimum of 384 - - The amount of off-heap memory (in megabytes) to be allocated per executor. This is memory that accounts for things like VM overheads, interned strings, other native overheads, etc. This tends to grow with the executor size (typically 6-10%). - - spark.yarn.am.memoryOverhead AM memory * 0.10, with minimum of 384 diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java index 4ef33ebd31a6a..c57af92029460 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java @@ -114,7 +114,7 @@ class SparkSubmitOptionParser { { QUEUE }, { REPOSITORIES }, { STATUS }, - { TOTAL_EXECUTOR_CORES } + { TOTAL_EXECUTOR_CORES }, }; /** diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index fd5dd7d7ff322..960277c90e1e8 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -81,17 +81,6 @@ private[spark] object Config extends Logging { .stringConf .createOptional - // Note that while we set a default for this when we start up the - // scheduler, the specific default value is dynamically determined - // based on the executor memory. - val KUBERNETES_EXECUTOR_MEMORY_OVERHEAD = - ConfigBuilder("spark.kubernetes.executor.memoryOverhead") - .doc("The amount of off-heap memory (in megabytes) to be allocated per executor. This " + - "is memory that accounts for things like VM overheads, interned strings, other native " + - "overheads, etc. This tends to grow with the executor size. (typically 6-10%).") - .bytesConf(ByteUnit.MiB) - .createOptional - val KUBERNETES_DRIVER_POD_NAME = ConfigBuilder("spark.kubernetes.driver.pod.name") .doc("Name of the driver pod.") diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala index d55f75faf657a..c563fc5bfbadf 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala @@ -91,7 +91,7 @@ private[spark] class DriverConfigurationStepsOrchestrator( val mayBeResource = mainAppResource.get match { case JavaMainAppResource(resource) if resource != SparkLauncher.NO_RESOURCE => Some(resource) - case _ => Option.empty + case _ => None } mayBeResource } else { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala index f79155b117b67..9d8f3b912c33d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala @@ -24,6 +24,7 @@ import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.ConfigurationUtils import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.internal.config.{EXECUTOR_CLASS_PATH, EXECUTOR_JAVA_OPTIONS, EXECUTOR_MEMORY, EXECUTOR_MEMORY_OVERHEAD} import org.apache.spark.util.Utils /** @@ -46,8 +47,7 @@ private[spark] trait ExecutorPodFactory { private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf) extends ExecutorPodFactory { - private val executorExtraClasspath = - sparkConf.get(org.apache.spark.internal.config.EXECUTOR_CLASS_PATH) + private val executorExtraClasspath = sparkConf.get(EXECUTOR_CLASS_PATH) private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs( sparkConf, @@ -81,13 +81,12 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf) private val executorPodNamePrefix = sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX) - private val executorMemoryMiB = sparkConf.get(org.apache.spark.internal.config.EXECUTOR_MEMORY) + private val executorMemoryMiB = sparkConf.get(EXECUTOR_MEMORY) private val executorMemoryString = sparkConf.get( - org.apache.spark.internal.config.EXECUTOR_MEMORY.key, - org.apache.spark.internal.config.EXECUTOR_MEMORY.defaultValueString) + EXECUTOR_MEMORY.key, EXECUTOR_MEMORY.defaultValueString) private val memoryOverheadMiB = sparkConf - .get(KUBERNETES_EXECUTOR_MEMORY_OVERHEAD) + .get(EXECUTOR_MEMORY_OVERHEAD) .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * executorMemoryMiB).toInt, MEMORY_OVERHEAD_MIN_MIB)) private val executorMemoryWithOverhead = executorMemoryMiB + memoryOverheadMiB @@ -129,7 +128,7 @@ private[spark] class ExecutorPodFactoryImpl(sparkConf: SparkConf) .build() } val executorExtraJavaOptionsEnv = sparkConf - .get(org.apache.spark.internal.config.EXECUTOR_JAVA_OPTIONS) + .get(EXECUTOR_JAVA_OPTIONS) .map { opts => val delimitedOpts = Utils.splitCommandString(opts) delimitedOpts.zipWithIndex.map { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala index 31d65b95e21da..b8bb152d17910 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala @@ -20,7 +20,7 @@ import java.io.File import io.fabric8.kubernetes.client.Config -import org.apache.spark.SparkContext +import org.apache.spark.{SparkContext, SparkException} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.SparkKubernetesClientFactory @@ -33,6 +33,10 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit override def canCreate(masterURL: String): Boolean = masterURL.startsWith("k8s") override def createTaskScheduler(sc: SparkContext, masterURL: String): TaskScheduler = { + if (masterURL.startsWith("k8s") && sc.deployMode == "client") { + throw new SparkException("Client mode is currently not supported for Kubernetes.") + } + new TaskSchedulerImpl(sc) } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala index d401280e6bdbc..3ba3ae5ab4401 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala @@ -223,10 +223,6 @@ package object config { .intConf .createWithDefault(1) - private[spark] val EXECUTOR_MEMORY_OVERHEAD = ConfigBuilder("spark.yarn.executor.memoryOverhead") - .bytesConf(ByteUnit.MiB) - .createOptional - private[spark] val EXECUTOR_NODE_LABEL_EXPRESSION = ConfigBuilder("spark.yarn.executor.nodeLabelExpression") .doc("Node label expression for executors.") From caf22060f600b3b382e2e98b7ee5f0aacc165f2d Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Fri, 8 Dec 2017 17:24:32 -0800 Subject: [PATCH 35/37] Addressed docs comments --- docs/configuration.md | 6 ++++-- .../src/main/scala/org/apache/spark/deploy/k8s/Config.scala | 4 ++-- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index 3d72d0dd9fabb..68e8158c7fbe1 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -163,7 +163,8 @@ of the most common options to set are: The amount of off-heap memory (in megabytes) to be allocated per driver in cluster mode. This is memory that accounts for things like VM overheads, interned strings, other native overheads, etc. - This tends to grow with the container size (typically 6-10%). + This tends to grow with the container size (typically 6-10%). This option is supported in Yarn + mode and Kubernetes mode. @@ -179,7 +180,8 @@ of the most common options to set are: The amount of off-heap memory (in megabytes) to be allocated per executor. This is memory that accounts for things like VM overheads, interned strings, other native overheads, etc. This tends - to grow with the executor size (typically 6-10%). + to grow with the executor size (typically 6-10%). This option is supported in Yarn mode and + Kubernetes mode. diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index 960277c90e1e8..f35fb38798218 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -131,7 +131,7 @@ private[spark] object Config extends Logging { .checkValue(interval => interval > 0, s"Logging interval must be a positive time value.") .createWithDefaultString("1s") - private[spark] val JARS_DOWNLOAD_LOCATION = + val JARS_DOWNLOAD_LOCATION = ConfigBuilder("spark.kubernetes.mountDependencies.jarsDownloadDir") .doc("Location to download jars to in the driver and executors. When using" + " spark-submit, this directory must be empty and will be mounted as an empty directory" + @@ -139,7 +139,7 @@ private[spark] object Config extends Logging { .stringConf .createWithDefault("/var/spark-data/spark-jars") - private[spark] val FILES_DOWNLOAD_LOCATION = + val FILES_DOWNLOAD_LOCATION = ConfigBuilder("spark.kubernetes.mountDependencies.filesDownloadDir") .doc("Location to download files to in the driver and executors. When using" + " spark-submit, this directory must be empty and will be mounted as an empty directory" + From 2e7810b49f00bbdb0cd5f4ac5cf99a74c748cf5a Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Sun, 10 Dec 2017 19:50:25 -0800 Subject: [PATCH 36/37] Fixed a comment --- core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 930af3103f26c..ab834bb682041 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -300,7 +300,7 @@ object SparkSubmit extends CommandLineUtils with Logging { if (clusterManager == KUBERNETES) { args.master = Utils.checkAndGetK8sMasterUrl(args.master) - // Make sure YARN is included in our build if we're trying to use it + // Make sure KUBERNETES is included in our build if we're trying to use it if (!Utils.classIsLoadable(KUBERNETES_CLUSTER_SUBMIT_CLASS) && !Utils.isTesting) { printErrorAndExit( "Could not load KUBERNETES classes. " + From cbcd30ea0a5eb75ed831f50707bc97035c0a3371 Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Mon, 11 Dec 2017 11:34:39 -0800 Subject: [PATCH 37/37] Addressed latest comments --- .../src/test/scala/org/apache/spark/util/UtilsSuite.scala | 8 ++++---- docs/configuration.md | 8 ++++---- 2 files changed, 8 insertions(+), 8 deletions(-) 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 18c9adccd7c59..5c4e4ca0cded6 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -1148,16 +1148,16 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { test("check Kubernetes master URL") { val k8sMasterURLHttps = Utils.checkAndGetK8sMasterUrl("k8s://https://host:port") - assert(k8sMasterURLHttps == "k8s:https://host:port") + assert(k8sMasterURLHttps === "k8s:https://host:port") val k8sMasterURLHttp = Utils.checkAndGetK8sMasterUrl("k8s://http://host:port") - assert(k8sMasterURLHttp == "k8s:http://host:port") + assert(k8sMasterURLHttp === "k8s:http://host:port") val k8sMasterURLWithoutScheme = Utils.checkAndGetK8sMasterUrl("k8s://127.0.0.1:8443") - assert(k8sMasterURLWithoutScheme == "k8s:https://127.0.0.1:8443") + assert(k8sMasterURLWithoutScheme === "k8s:https://127.0.0.1:8443") val k8sMasterURLWithoutScheme2 = Utils.checkAndGetK8sMasterUrl("k8s://127.0.0.1") - assert(k8sMasterURLWithoutScheme2 == "k8s:https://127.0.0.1") + assert(k8sMasterURLWithoutScheme2 === "k8s:https://127.0.0.1") intercept[IllegalArgumentException] { Utils.checkAndGetK8sMasterUrl("k8s:https://host:port") diff --git a/docs/configuration.md b/docs/configuration.md index 68e8158c7fbe1..d70bac134808f 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -163,8 +163,8 @@ of the most common options to set are: The amount of off-heap memory (in megabytes) to be allocated per driver in cluster mode. This is memory that accounts for things like VM overheads, interned strings, other native overheads, etc. - This tends to grow with the container size (typically 6-10%). This option is supported in Yarn - mode and Kubernetes mode. + This tends to grow with the container size (typically 6-10%). This option is currently supported + on YARN and Kubernetes. @@ -180,8 +180,8 @@ of the most common options to set are: The amount of off-heap memory (in megabytes) to be allocated per executor. This is memory that accounts for things like VM overheads, interned strings, other native overheads, etc. This tends - to grow with the executor size (typically 6-10%). This option is supported in Yarn mode and - Kubernetes mode. + to grow with the executor size (typically 6-10%). This option is currently supported on YARN and + Kubernetes.